diff --git a/core/Cargo.lock b/core/Cargo.lock index 59c46c92be..723d5ef1f2 100644 --- a/core/Cargo.lock +++ b/core/Cargo.lock @@ -144,7 +144,7 @@ dependencies = [ "arrow-data", "arrow-schema", "chrono", - "half 2.4.1", + "half", "num", ] @@ -160,7 +160,7 @@ dependencies = [ "arrow-schema", "chrono", "chrono-tz 0.9.0", - "half 2.4.1", + "half", "hashbrown", "num", ] @@ -172,7 +172,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fe9b95e825ae838efaf77e366c00d3fc8cca78134c9db497d6bda425f2e7b7c1" dependencies = [ "bytes", - "half 2.4.1", + "half", "num", ] @@ -191,7 +191,7 @@ dependencies = [ "base64", "chrono", "comfy-table", - "half 2.4.1", + "half", "lexical-core", "num", "ryu", @@ -224,7 +224,7 @@ checksum = "cb29be98f987bcf217b070512bb7afba2f65180858bca462edf4a39d84a23e10" dependencies = [ "arrow-buffer", "arrow-schema", - "half 2.4.1", + "half", "num", ] @@ -255,7 +255,7 @@ dependencies = [ "arrow-data", "arrow-schema", "chrono", - "half 2.4.1", + "half", "indexmap", "lexical-core", "num", @@ -274,7 +274,7 @@ dependencies = [ "arrow-data", "arrow-schema", "arrow-select", - "half 2.4.1", + "half", "num", ] @@ -289,7 +289,7 @@ dependencies = [ "arrow-buffer", "arrow-data", "arrow-schema", - "half 2.4.1", + "half", "hashbrown", ] @@ -482,9 +482,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.0.98" +version = "1.0.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41c270e7540d725e65ac7f1b212ac8ce349719624d7bcff99f8e2e488e8cf03f" +checksum = "96c51067fd44124faa7f870b4b1c969379ad32b2ba805aa959430ceaa384f695" dependencies = [ "jobserver", "libc", @@ -561,9 +561,9 @@ dependencies = [ [[package]] name = "ciborium" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "effd91f6c78e5a4ace8a5d3c0b6bfaec9e2baaef55f3efc00e45fb2e477ee926" +checksum = "42e69ffd6f0917f5c029256a24d0161db17cea3997d185db0d35926308770f0e" dependencies = [ "ciborium-io", "ciborium-ll", @@ -578,28 +578,28 @@ checksum = "05afea1e0a06c9be33d539b876f1ce3692f4afea2cb41f740e7743225ed1c757" [[package]] name = "ciborium-ll" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "defaa24ecc093c77630e6c15e17c51f5e187bf35ee514f4e2d67baaa96dae22b" +checksum = "57663b653d948a338bfb3eeba9bb2fd5fcfaecb9e199e87e1eda4d9e8b240fd9" dependencies = [ "ciborium-io", - "half 1.8.3", + "half", ] [[package]] name = "clap" -version = "4.5.4" +version = "4.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90bc066a67923782aa8515dbaea16946c5bcc5addbd668bb80af688e53e548a0" +checksum = "a9689a29b593160de5bc4aacab7b5d54fb52231de70122626c178e6a368994c7" dependencies = [ "clap_builder", ] [[package]] name = "clap_builder" -version = "4.5.2" +version = "4.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae129e2e766ae0ec03484e609954119f123cc1fe650337e155d03b022f24f7b4" +checksum = "2e5387378c84f6faa26890ebf9f0a92989f8873d4d380467bcd0d8d8620424df" dependencies = [ "anstyle", "clap_lex", @@ -607,9 +607,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.7.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98cc8fbded0c607b7ba9dd60cd98df59af97e84d24e49c8557331cfc26d301ce" +checksum = "4b82cf0babdbd58558212896d1a4272303a57bdb245c2bf1147185fb45640e70" [[package]] name = "combine" @@ -649,7 +649,7 @@ dependencies = [ "datafusion-physical-expr-common", "flate2", "futures", - "half 2.4.1", + "half", "hashbrown", "itertools 0.11.0", "jni", @@ -890,7 +890,7 @@ dependencies = [ "datafusion-sql", "futures", "glob", - "half 2.4.1", + "half", "hashbrown", "indexmap", "itertools 0.12.1", @@ -919,7 +919,7 @@ dependencies = [ "arrow-buffer", "arrow-schema", "chrono", - "half 2.4.1", + "half", "hashbrown", "instant", "libc", @@ -1054,7 +1054,7 @@ dependencies = [ "datafusion-expr", "datafusion-functions-aggregate", "datafusion-physical-expr-common", - "half 2.4.1", + "half", "hashbrown", "hex", "indexmap", @@ -1097,7 +1097,7 @@ dependencies = [ "datafusion-physical-expr", "datafusion-physical-expr-common", "futures", - "half 2.4.1", + "half", "hashbrown", "indexmap", "itertools 0.12.1", @@ -1371,12 +1371,6 @@ version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" -[[package]] -name = "half" -version = "1.8.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b43ede17f21864e81be2fa654110bf1e793774238d86ef8555c37e6519c0403" - [[package]] name = "half" version = "2.4.1" @@ -2063,7 +2057,7 @@ dependencies = [ "ahash", "bytes", "chrono", - "half 2.4.1", + "half", "hashbrown", "num", "num-bigint", @@ -2661,9 +2655,9 @@ dependencies = [ [[package]] name = "strum_macros" -version = "0.26.3" +version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7993a8e3a9e88a00351486baae9522c91b123a088f76469e5bd5cc17198ea87" +checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" dependencies = [ "heck 0.5.0", "proc-macro2", @@ -2680,9 +2674,9 @@ checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" [[package]] name = "symbolic-common" -version = "12.8.0" +version = "12.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cccfffbc6bb3bb2d3a26cd2077f4d055f6808d266f9d4d158797a4c60510dfe" +checksum = "71297dc3e250f7dbdf8adb99e235da783d690f5819fdeb4cce39d9cfb0aca9f1" dependencies = [ "debugid", "memmap2", @@ -2692,9 +2686,9 @@ dependencies = [ [[package]] name = "symbolic-demangle" -version = "12.8.0" +version = "12.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76a99812da4020a67e76c4eb41f08c87364c14170495ff780f30dd519c221a68" +checksum = "424fa2c9bf2c862891b9cfd354a752751a6730fd838a4691e7f6c2c7957b9daf" dependencies = [ "cpp_demangle", "rustc-demangle", @@ -2955,9 +2949,9 @@ checksum = "d4c87d22b6e3f4a18d4d40ef354e97c90fcb14dd91d7dc0aa9d8a1172ebf7202" [[package]] name = "unicode-width" -version = "0.1.12" +version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68f5e5f3158ecfd4b8ff6fe086db7c8467a2dfdac97fe420f2b7c4aa97af66d6" +checksum = "0336d538f7abc86d282a4189614dfaa90810dfc2c6f6427eaf88e16311dd225d" [[package]] name = "unsafe-any-ors" diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index 6c7ea0de48..fcef182a78 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -35,6 +35,7 @@ use datafusion::{ }, AggregateExpr, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, }, + physical_optimizer::join_selection::swap_hash_join, physical_plan::{ aggregates::{AggregateMode as DFAggregateMode, PhysicalGroupBy}, filter::FilterExec, @@ -91,7 +92,7 @@ use crate::{ agg_expr::ExprStruct as AggExprStruct, expr::ExprStruct, literal::Value, AggExpr, Expr, ScalarFunc, }, - spark_operator::{operator::OpStruct, JoinType, Operator}, + spark_operator::{operator::OpStruct, BuildSide, JoinType, Operator}, spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }, }; @@ -965,7 +966,7 @@ impl PhysicalPlanner { join.join_type, &join.condition, )?; - let join = Arc::new(HashJoinExec::try_new( + let hash_join = Arc::new(HashJoinExec::try_new( join_params.left, join_params.right, join_params.join_on, @@ -977,7 +978,15 @@ impl PhysicalPlanner { // `EqualNullSafe`, Spark will rewrite it during planning. false, )?); - Ok((scans, join)) + + // If the hash join is build right, we need to swap the left and right + let hash_join = if join.build_side == BuildSide::BuildLeft as i32 { + hash_join + } else { + swap_hash_join(hash_join.as_ref(), PartitionMode::Partitioned)? + }; + + Ok((scans, hash_join)) } } } diff --git a/core/src/execution/proto/operator.proto b/core/src/execution/proto/operator.proto index 6080c56682..de25f94dae 100644 --- a/core/src/execution/proto/operator.proto +++ b/core/src/execution/proto/operator.proto @@ -95,6 +95,7 @@ message HashJoin { repeated spark.spark_expression.Expr right_join_keys = 2; JoinType join_type = 3; optional spark.spark_expression.Expr condition = 4; + BuildSide build_side = 5; } message SortMergeJoin { @@ -114,3 +115,8 @@ enum JoinType { LeftAnti = 6; RightAnti = 7; } + +enum BuildSide { + BuildLeft = 0; + BuildRight = 1; +} diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index c6d10c2ec9..8129c74557 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -352,7 +352,7 @@ index daef11ae4d6..9f3cc9181f2 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..060f874ea72 100644 +index f33432ddb6f..9cf7a9dd4e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -373,7 +373,37 @@ index f33432ddb6f..060f874ea72 100644 case _ => Nil } } -@@ -1187,7 +1191,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -665,7 +669,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("partition pruning in broadcast hash joins with aliases") { ++ test("partition pruning in broadcast hash joins with aliases", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + Given("alias with simple join condition, using attribute names only") + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( +@@ -755,7 +760,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("partition pruning in broadcast hash joins") { ++ test("partition pruning in broadcast hash joins", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + Given("disable broadcast pruning and disable subquery duplication") + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", +@@ -990,7 +996,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("different broadcast subqueries with identical children") { ++ test("different broadcast subqueries with identical children", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + withTable("fact", "dim") { + spark.range(100).select( +@@ -1187,7 +1194,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -383,7 +413,7 @@ index f33432ddb6f..060f874ea72 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ -@@ -1238,7 +1243,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1238,7 +1246,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -393,7 +423,27 @@ index f33432ddb6f..060f874ea72 100644 Given("dynamic pruning filter on the build side") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( -@@ -1485,7 +1491,7 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1311,7 +1320,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("SPARK-32817: DPP throws error when the broadcast side is empty") { ++ test("SPARK-32817: DPP throws error when the broadcast side is empty", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", +@@ -1470,7 +1480,8 @@ abstract class DynamicPartitionPruningSuiteBase + checkAnswer(df, Row(3, 2) :: Row(3, 2) :: Row(3, 2) :: Row(3, 2) :: Nil) + } + +- test("SPARK-36444: Remove OptimizeSubqueries from batch of PartitionPruning") { ++ test("SPARK-36444: Remove OptimizeSubqueries from batch of PartitionPruning", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + val df = sql( + """ +@@ -1485,7 +1496,7 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-38148: Do not add dynamic partition pruning if there exists static partition " + @@ -402,7 +452,37 @@ index f33432ddb6f..060f874ea72 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { Seq( "f.store_id = 1" -> false, -@@ -1729,6 +1735,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1557,7 +1568,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") { ++ test("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withTable("duplicate_keys") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + Seq[(Int, String)]((1, "NL"), (1, "NL"), (3, "US"), (3, "US"), (3, "US")) +@@ -1588,7 +1600,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("SPARK-39338: Remove dynamic pruning subquery if pruningKey's references is empty") { ++ test("SPARK-39338: Remove dynamic pruning subquery if pruningKey's references is empty", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + val df = sql( + """ +@@ -1617,7 +1630,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("SPARK-39217: Makes DPP support the pruning side has Union") { ++ test("SPARK-39217: Makes DPP support the pruning side has Union", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + val df = sql( + """ +@@ -1729,6 +1743,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -966,13 +1046,15 @@ index 4b3d3a4b805..56e1e0e6f16 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -index 9e9d717db3b..91a4f9a38d5 100644 +index 9e9d717db3b..c1a7caf56e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -@@ -18,6 +18,7 @@ +@@ -17,7 +17,8 @@ + package org.apache.spark.sql.execution - import org.apache.spark.sql.{DataFrame, QueryTest, Row} +-import org.apache.spark.sql.{DataFrame, QueryTest, Row} ++import org.apache.spark.sql.{DataFrame, IgnoreComet, QueryTest, Row} +import org.apache.spark.sql.comet.CometProjectExec import org.apache.spark.sql.connector.SimpleWritableDataSource import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} @@ -989,6 +1071,16 @@ index 9e9d717db3b..91a4f9a38d5 100644 assert(actual == expected) } } +@@ -112,7 +116,8 @@ abstract class RemoveRedundantProjectsSuiteBase + assertProjectExec(query, 1, 3) + } + +- test("join with ordering requirement") { ++ test("join with ordering requirement", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + val query = "select * from (select key, a, c, b from testView) as t1 join " + + "(select key, a, b, c from testView) as t2 on t1.key = t2.key where t2.a > 50" + assertProjectExec(query, 2, 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala index 30ce940b032..0d3f6c6c934 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index ed3f2fae61..7d9bef48c4 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -25,7 +25,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Average, BitAndAgg, BitOrAgg, BitXorAgg, Corr, Count, CovPopulation, CovSample, Final, First, Last, Max, Min, Partial, StddevPop, StddevSamp, Sum, VariancePop, VarianceSamp} import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke -import org.apache.spark.sql.catalyst.optimizer.{BuildRight, NormalizeNaNAndZero} +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, NormalizeNaNAndZero} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, SinglePartition} import org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils @@ -46,7 +46,7 @@ import org.apache.comet.CometSparkSessionExtensions.{isCometOperatorEnabled, isC import org.apache.comet.expressions.{CometCast, CometEvalMode, Compatible, Incompatible, Unsupported} import org.apache.comet.serde.ExprOuterClass.{AggExpr, DataType => ProtoDataType, Expr, ScalarFunc} import org.apache.comet.serde.ExprOuterClass.DataType.{DataTypeInfo, DecimalInfo, ListInfo, MapInfo, StructInfo} -import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, JoinType, Operator} +import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, BuildSide, JoinType, Operator} import org.apache.comet.shims.CometExprShim import org.apache.comet.shims.ShimQueryPlanSerde @@ -2468,10 +2468,8 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim return None } - if (join.buildSide == BuildRight) { - // DataFusion HashJoin assumes build side is always left. - // TODO: support BuildRight - withInfo(join, "BuildRight is not supported") + if (join.buildSide == BuildRight && join.joinType == LeftAnti) { + withInfo(join, "BuildRight with LeftAnti is not supported") return None } @@ -2508,6 +2506,8 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim .setJoinType(joinType) .addAllLeftJoinKeys(leftKeys.map(_.get).asJava) .addAllRightJoinKeys(rightKeys.map(_.get).asJava) + .setBuildSide( + if (join.buildSide == BuildLeft) BuildSide.BuildLeft else BuildSide.BuildRight) condition.foreach(joinBuilder.setCondition) Some(result.setHashJoin(joinBuilder).build()) } else { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index bb17442a59..394e4b1560 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -702,7 +702,7 @@ case class CometHashJoinExec( this.copy(left = newLeft, right = newRight) override def stringArgs: Iterator[Any] = - Iterator(leftKeys, rightKeys, joinType, condition, left, right) + Iterator(leftKeys, rightKeys, joinType, buildSide, condition, left, right) override def equals(obj: Any): Boolean = { obj match { @@ -836,7 +836,7 @@ case class CometBroadcastHashJoinExec( this.copy(left = newLeft, right = newRight) override def stringArgs: Iterator[Any] = - Iterator(leftKeys, rightKeys, joinType, condition, left, right) + Iterator(leftKeys, rightKeys, joinType, condition, buildSide, left, right) override def equals(obj: Any): Boolean = { obj match { diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt index 762f3a4f6a..f3cd64a14c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt @@ -1,44 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Filter (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_returns (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (24) - : : +- * Filter (23) - : : +- * HashAggregate (22) - : : +- Exchange (21) - : : +- * HashAggregate (20) - : : +- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * ColumnarToRow (13) - : : : +- CometFilter (12) - : : : +- CometScan parquet spark_catalog.default.store_returns (11) - : : +- ReusedExchange (14) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.store (27) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer (34) +TakeOrderedAndProject (43) ++- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Filter (13) + : : : +- * HashAggregate (12) + : : : +- Exchange (11) + : : : +- * HashAggregate (10) + : : : +- * ColumnarToRow (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_returns (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (27) + : : +- * Filter (26) + : : +- * HashAggregate (25) + : : +- Exchange (24) + : : +- * HashAggregate (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * ColumnarToRow (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.store_returns (14) + : : +- ReusedExchange (16) + : +- BroadcastExchange (34) + : +- * ColumnarToRow (33) + : +- CometProject (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.store (30) + +- BroadcastExchange (40) + +- * ColumnarToRow (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.customer (37) (1) Scan parquet spark_catalog.default.store_returns @@ -53,222 +56,236 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_year#7] +Arguments: [d_date_sk#6], [d_date_sk#6] -(6) Project [codegen id : 2] -Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] + +(7) CometBroadcastHashJoin +Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] +Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] + +(9) ColumnarToRow [codegen id : 1] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -(7) HashAggregate [codegen id : 2] +(10) HashAggregate [codegen id : 1] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] +Aggregate Attributes [1]: [sum#8] +Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#9] -(8) Exchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] +(11) Exchange +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#9] Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) HashAggregate [codegen id : 9] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] +(12) HashAggregate [codegen id : 7] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#9] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] -Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#10] +Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#11, sr_store_sk#2 AS ctr_store_sk#12, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#10,17,2) AS ctr_total_return#13] -(10) Filter [codegen id : 9] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] -Condition : isnotnull(ctr_total_return#12) +(13) Filter [codegen id : 7] +Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13] +Condition : isnotnull(ctr_total_return#13) -(11) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] +(14) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(sr_returned_date_sk#17 IN dynamicpruning#18)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(12) CometFilter -Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -Condition : isnotnull(sr_store_sk#14) +(15) CometFilter +Input [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +Condition : isnotnull(sr_store_sk#15) -(13) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] +(16) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#19] -(14) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#18] +(17) CometBroadcastHashJoin +Left output [4]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [sr_returned_date_sk#17], [d_date_sk#19], Inner, BuildRight -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#16] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None +(18) CometProject +Input [5]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16, sr_returned_date_sk#17, d_date_sk#19] +Arguments: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16], [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] + +(19) ColumnarToRow [codegen id : 2] +Input [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -(16) Project [codegen id : 4] -Output [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] -Input [5]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16, d_date_sk#18] - -(17) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] -Keys [2]: [sr_customer_sk#13, sr_store_sk#14] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#15))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [sr_customer_sk#13, sr_store_sk#14, sum#20] - -(18) Exchange -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#20] -Arguments: hashpartitioning(sr_customer_sk#13, sr_store_sk#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(19) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#20] -Keys [2]: [sr_customer_sk#13, sr_store_sk#14] -Functions [1]: [sum(UnscaledValue(sr_return_amt#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#15))#9] -Results [2]: [sr_store_sk#14 AS ctr_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#15))#9,17,2) AS ctr_total_return#22] - -(20) HashAggregate [codegen id : 5] -Input [2]: [ctr_store_sk#21, ctr_total_return#22] -Keys [1]: [ctr_store_sk#21] -Functions [1]: [partial_avg(ctr_total_return#22)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ctr_store_sk#21, sum#25, count#26] +(20) HashAggregate [codegen id : 2] +Input [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] +Keys [2]: [sr_customer_sk#14, sr_store_sk#15] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#16))] +Aggregate Attributes [1]: [sum#20] +Results [3]: [sr_customer_sk#14, sr_store_sk#15, sum#21] (21) Exchange -Input [3]: [ctr_store_sk#21, sum#25, count#26] -Arguments: hashpartitioning(ctr_store_sk#21, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(22) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#21, sum#25, count#26] -Keys [1]: [ctr_store_sk#21] -Functions [1]: [avg(ctr_total_return#22)] -Aggregate Attributes [1]: [avg(ctr_total_return#22)#27] -Results [2]: [(avg(ctr_total_return#22)#27 * 1.2) AS (avg(ctr_total_return) * 1.2)#28, ctr_store_sk#21] - -(23) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_store_sk#21] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#28) - -(24) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_store_sk#21] +Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#21] +Arguments: hashpartitioning(sr_customer_sk#14, sr_store_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(22) HashAggregate [codegen id : 3] +Input [3]: [sr_customer_sk#14, sr_store_sk#15, sum#21] +Keys [2]: [sr_customer_sk#14, sr_store_sk#15] +Functions [1]: [sum(UnscaledValue(sr_return_amt#16))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#16))#10] +Results [2]: [sr_store_sk#15 AS ctr_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#10,17,2) AS ctr_total_return#23] + +(23) HashAggregate [codegen id : 3] +Input [2]: [ctr_store_sk#22, ctr_total_return#23] +Keys [1]: [ctr_store_sk#22] +Functions [1]: [partial_avg(ctr_total_return#23)] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ctr_store_sk#22, sum#26, count#27] + +(24) Exchange +Input [3]: [ctr_store_sk#22, sum#26, count#27] +Arguments: hashpartitioning(ctr_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(25) HashAggregate [codegen id : 4] +Input [3]: [ctr_store_sk#22, sum#26, count#27] +Keys [1]: [ctr_store_sk#22] +Functions [1]: [avg(ctr_total_return#23)] +Aggregate Attributes [1]: [avg(ctr_total_return#23)#28] +Results [2]: [(avg(ctr_total_return#23)#28 * 1.2) AS (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#22] + +(26) Filter [codegen id : 4] +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#22] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) + +(27) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_store_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [ctr_store_sk#21] +(28) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ctr_store_sk#12] +Right keys [1]: [ctr_store_sk#22] Join type: Inner -Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#28) +Join condition: (cast(ctr_total_return#13 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) -(26) Project [codegen id : 9] -Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] -Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#28, ctr_store_sk#21] +(29) Project [codegen id : 7] +Output [2]: [ctr_customer_sk#11, ctr_store_sk#12] +Input [5]: [ctr_customer_sk#11, ctr_store_sk#12, ctr_total_return#13, (avg(ctr_total_return) * 1.2)#29, ctr_store_sk#22] -(27) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#29, s_state#30] +(30) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#30, s_state#31] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [s_store_sk#29, s_state#30] -Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) +(31) CometFilter +Input [2]: [s_store_sk#30, s_state#31] +Condition : ((isnotnull(s_state#31) AND (s_state#31 = TN)) AND isnotnull(s_store_sk#30)) -(29) CometProject -Input [2]: [s_store_sk#29, s_state#30] -Arguments: [s_store_sk#29], [s_store_sk#29] +(32) CometProject +Input [2]: [s_store_sk#30, s_state#31] +Arguments: [s_store_sk#30], [s_store_sk#30] -(30) ColumnarToRow [codegen id : 7] -Input [1]: [s_store_sk#29] +(33) ColumnarToRow [codegen id : 5] +Input [1]: [s_store_sk#30] -(31) BroadcastExchange -Input [1]: [s_store_sk#29] +(34) BroadcastExchange +Input [1]: [s_store_sk#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [s_store_sk#29] +(35) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ctr_store_sk#12] +Right keys [1]: [s_store_sk#30] Join type: Inner Join condition: None -(33) Project [codegen id : 9] -Output [1]: [ctr_customer_sk#10] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#29] +(36) Project [codegen id : 7] +Output [1]: [ctr_customer_sk#11] +Input [3]: [ctr_customer_sk#11, ctr_store_sk#12, s_store_sk#30] -(34) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#31, c_customer_id#32] +(37) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#32, c_customer_id#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [c_customer_sk#31, c_customer_id#32] -Condition : isnotnull(c_customer_sk#31) +(38) CometFilter +Input [2]: [c_customer_sk#32, c_customer_id#33] +Condition : isnotnull(c_customer_sk#32) -(36) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#31, c_customer_id#32] +(39) ColumnarToRow [codegen id : 6] +Input [2]: [c_customer_sk#32, c_customer_id#33] -(37) BroadcastExchange -Input [2]: [c_customer_sk#31, c_customer_id#32] +(40) BroadcastExchange +Input [2]: [c_customer_sk#32, c_customer_id#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_customer_sk#10] -Right keys [1]: [c_customer_sk#31] +(41) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ctr_customer_sk#11] +Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(39) Project [codegen id : 9] -Output [1]: [c_customer_id#32] -Input [3]: [ctr_customer_sk#10, c_customer_sk#31, c_customer_id#32] +(42) Project [codegen id : 7] +Output [1]: [c_customer_id#33] +Input [3]: [ctr_customer_sk#11, c_customer_sk#32, c_customer_id#33] -(40) TakeOrderedAndProject -Input [1]: [c_customer_id#32] -Arguments: 100, [c_customer_id#32 ASC NULLS FIRST], [c_customer_id#32] +(43) TakeOrderedAndProject +Input [1]: [c_customer_id#33] +Arguments: 100, [c_customer_id#33 ASC NULLS FIRST], [c_customer_id#33] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(41) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#33] +(44) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_year#7] 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 -(42) CometFilter -Input [2]: [d_date_sk#6, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2000)) AND isnotnull(d_date_sk#6)) +(45) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(43) CometProject -Input [2]: [d_date_sk#6, d_year#33] +(46) CometProject +Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(44) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(45) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 14 Hosting Expression = sr_returned_date_sk#17 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt index 688fb69a8e..26b5570b72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (9) + WholeStageCodegen (7) Project [c_customer_id] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk] @@ -10,12 +10,12 @@ TakeOrderedAndProject [c_customer_id] HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] InputAdapter Exchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometFilter [sr_store_sk,sr_customer_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -26,42 +26,43 @@ TakeOrderedAndProject [c_customer_id] CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (6) + BroadcastExchange #4 + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] InputAdapter - Exchange [ctr_store_sk] #4 - WholeStageCodegen (5) + Exchange [ctr_store_sk] #5 + WholeStageCodegen (3) HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] InputAdapter - Exchange [sr_customer_sk,sr_store_sk] #5 - WholeStageCodegen (4) + Exchange [sr_customer_sk,sr_store_sk] #6 + WholeStageCodegen (2) HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [s_store_sk] CometFilter [s_state,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #8 + WholeStageCodegen (6) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt index 15490b87d7..4a29b72602 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt @@ -1,47 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer_demographics (34) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * Filter (27) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) + : : : :- * ColumnarToRow (12) + : : : : +- CometBroadcastHashJoin (11) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (10) + : : : : +- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (33) + : +- * ColumnarToRow (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (39) + +- * ColumnarToRow (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.customer_demographics (36) (1) Scan parquet spark_catalog.default.customer @@ -55,232 +57,241 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#9] +(5) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(6) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(7) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] + +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] +(9) CometProject Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ss_customer_sk#6], [ss_customer_sk#6] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#6] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Right output [1]: [ss_customer_sk#6] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight + +(12) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(13) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#15] -(13) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#13] +(15) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +Right output [1]: [d_date_sk#15] +Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(16) CometProject +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] +Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +(17) ColumnarToRow [codegen id : 1] +Input [1]: [ws_bill_customer_sk#12] -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) BroadcastExchange +Input [1]: [ws_bill_customer_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 9] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] +Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#19] -(20) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#17] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(23) CometProject +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#14] -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] +(24) ColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#16] -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#14] +Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(27) Filter [codegen id : 5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(28) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(27) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_county#19] +(29) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [ca_address_sk#18, ca_county#19] -Condition : (ca_county#19 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#18)) +(30) CometFilter +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) -(29) CometProject -Input [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] +(31) CometProject +Input [2]: [ca_address_sk#20, ca_county#21] +Arguments: [ca_address_sk#20], [ca_address_sk#20] -(30) ColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#18] +(32) ColumnarToRow [codegen id : 3] +Input [1]: [ca_address_sk#20] -(31) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(33) BroadcastExchange +Input [1]: [ca_address_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(32) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(35) Project [codegen id : 5] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] -(34) Scan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(36) Scan parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(35) CometFilter -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#20) +(37) CometFilter +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#22) -(36) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(38) ColumnarToRow [codegen id : 4] +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(37) BroadcastExchange -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(39) BroadcastExchange +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(38) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#20] +Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(39) Project [codegen id : 9] -Output [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(41) Project [codegen id : 5] +Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(40) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(42) HashAggregate [codegen id : 5] +Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] +Aggregate Attributes [1]: [count#31] +Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32] -(41) Exchange -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(43) Exchange +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32] +Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(42) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(44) HashAggregate [codegen id : 6] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] +Aggregate Attributes [1]: [count(1)#33] +Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#33 AS cnt1#34, cd_purchase_estimate#26, count(1)#33 AS cnt2#35, cd_credit_rating#27, count(1)#33 AS cnt3#36, cd_dep_count#28, count(1)#33 AS cnt4#37, cd_dep_employed_count#29, count(1)#33 AS cnt5#38, cd_dep_college_count#30, count(1)#33 AS cnt6#39] -(43) TakeOrderedAndProject -Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] -Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] +(45) TakeOrderedAndProject +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#34, cd_purchase_estimate#26, cnt2#35, cd_credit_rating#27, cnt3#36, cd_dep_count#28, cnt4#37, cd_dep_employed_count#29, cnt5#38, cd_dep_college_count#30, cnt6#39] +Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#34, cd_purchase_estimate#26, cnt2#35, cd_credit_rating#27, cnt3#36, cd_dep_count#28, cnt4#37, cd_dep_employed_count#29, cnt5#38, cd_dep_college_count#30, cnt6#39] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(44) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#38, d_moy#39] +(46) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [3]: [d_date_sk#9, d_year#38, d_moy#39] -Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 1)) AND (d_moy#39 <= 4)) AND isnotnull(d_date_sk#9)) +(47) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) -(46) CometProject -Input [3]: [d_date_sk#9, d_year#38, d_moy#39] +(48) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(47) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(48) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt index 89893c831e..19243e359d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) + WholeStageCodegen (6) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,62 +13,58 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Filter [exists,exists] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [ca_address_sk] CometFilter [ca_county,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt index c663d46886..3f5dd24f2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt @@ -2,75 +2,75 @@ TakeOrderedAndProject (72) +- * Project (71) +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) - : :- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (51) - : +- * Filter (50) - : +- * HashAggregate (49) - : +- Exchange (48) - : +- * HashAggregate (47) - : +- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (43) - : : +- * BroadcastHashJoin Inner BuildRight (42) - : : :- * ColumnarToRow (37) - : : : +- CometFilter (36) - : : : +- CometScan parquet spark_catalog.default.customer (35) - : : +- BroadcastExchange (41) - : : +- * ColumnarToRow (40) - : : +- CometFilter (39) - : : +- CometScan parquet spark_catalog.default.web_sales (38) - : +- ReusedExchange (44) + :- * Project (54) + : +- * BroadcastHashJoin Inner BuildRight (53) + : :- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (17) + : : : +- * HashAggregate (16) + : : : +- Exchange (15) + : : : +- * HashAggregate (14) + : : : +- * ColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- BroadcastExchange (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * HashAggregate (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- CometBroadcastExchange (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- BroadcastExchange (52) + : +- * Filter (51) + : +- * HashAggregate (50) + : +- Exchange (49) + : +- * HashAggregate (48) + : +- * ColumnarToRow (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (43) + : : +- CometBroadcastHashJoin (42) + : : :- CometFilter (38) + : : : +- CometScan parquet spark_catalog.default.customer (37) + : : +- CometBroadcastExchange (41) + : : +- CometFilter (40) + : : +- CometScan parquet spark_catalog.default.web_sales (39) + : +- ReusedExchange (44) +- BroadcastExchange (69) +- * HashAggregate (68) +- Exchange (67) +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * ColumnarToRow (56) - : : +- CometFilter (55) - : : +- CometScan parquet spark_catalog.default.customer (54) - : +- BroadcastExchange (60) - : +- * ColumnarToRow (59) - : +- CometFilter (58) - : +- CometScan parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (63) + +- * ColumnarToRow (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometFilter (56) + : : +- CometScan parquet spark_catalog.default.customer (55) + : +- CometBroadcastExchange (59) + : +- CometFilter (58) + : +- CometScan parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (62) (1) Scan parquet spark_catalog.default.customer @@ -84,10 +84,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] Condition : isnotnull(ss_customer_sk#9) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Right output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: [c_customer_sk#1], [ss_customer_sk#9], Inner, BuildRight -(9) Project [codegen id : 3] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +(7) CometProject Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(10) ReusedExchange [Reuses operator id: 76] +(8) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_year#15] +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 -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) + +(10) CometBroadcastExchange +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14, d_year#15] -(12) Project [codegen id : 3] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] +(11) CometBroadcastHashJoin +Left output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Right output [2]: [d_date_sk#14, d_year#15] +Arguments: [ss_sold_date_sk#12], [d_date_sk#14], Inner, BuildRight + +(12) CometProject Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#14, d_year#15] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] + +(13) ColumnarToRow [codegen id : 1] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] -(13) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] Aggregate Attributes [1]: [sum#16] Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -(14) Exchange +(15) Exchange Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 16] +(16) HashAggregate [codegen id : 8] Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] -(16) Filter [codegen id : 16] +(17) Filter [codegen id : 8] Input [2]: [customer_id#19, year_total#20] Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) -(17) Scan parquet spark_catalog.default.customer +(18) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter +(19) CometFilter Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) -(19) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] - (20) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Batched: true @@ -177,83 +181,90 @@ ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(26) CometFilter +Input [2]: [d_date_sk#34, d_year#35] +Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) + +(27) CometBroadcastExchange +Input [2]: [d_date_sk#34, d_year#35] +Arguments: [d_date_sk#34, d_year#35] -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] +(28) CometBroadcastHashJoin +Left output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Right output [2]: [d_date_sk#34, d_year#35] +Arguments: [ss_sold_date_sk#32], [d_date_sk#34], Inner, BuildRight + +(29) CometProject Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] +Arguments: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35], [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] + +(30) ColumnarToRow [codegen id : 2] +Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] -(29) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] Aggregate Attributes [1]: [sum#36] Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -(30) Exchange +(32) Exchange Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(31) HashAggregate [codegen id : 7] +(33) HashAggregate [codegen id : 3] Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] Results [3]: [c_customer_id#22 AS customer_id#38, c_preferred_cust_flag#25 AS customer_preferred_cust_flag#39, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#40] -(32) BroadcastExchange +(34) BroadcastExchange Input [3]: [customer_id#38, customer_preferred_cust_flag#39, year_total#40] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 16] +(35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#38] Join type: Inner Join condition: None -(34) Project [codegen id : 16] +(36) Project [codegen id : 8] Output [4]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40] Input [5]: [customer_id#19, year_total#20, customer_id#38, customer_preferred_cust_flag#39, year_total#40] -(35) Scan parquet spark_catalog.default.customer +(37) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(36) CometFilter +(38) CometFilter Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_customer_id#42)) -(37) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] - -(38) Scan parquet spark_catalog.default.web_sales +(39) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] @@ -261,90 +272,85 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(39) CometFilter +(40) CometFilter Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] Condition : isnotnull(ws_bill_customer_sk#49) -(40) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] - -(41) BroadcastExchange +(41) CometBroadcastExchange Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -(42) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#41] -Right keys [1]: [ws_bill_customer_sk#49] -Join type: Inner -Join condition: None +(42) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Right output [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Arguments: [c_customer_sk#41], [ws_bill_customer_sk#49], Inner, BuildRight -(43) Project [codegen id : 10] -Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +(43) CometProject Input [12]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Arguments: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52], [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -(44) ReusedExchange [Reuses operator id: 76] +(44) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#54, d_year#55] -(45) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#54] -Join type: Inner -Join condition: None +(45) CometBroadcastHashJoin +Left output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Right output [2]: [d_date_sk#54, d_year#55] +Arguments: [ws_sold_date_sk#52], [d_date_sk#54], Inner, BuildRight -(46) Project [codegen id : 10] -Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] +(46) CometProject Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52, d_date_sk#54, d_year#55] +Arguments: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55], [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] -(47) HashAggregate [codegen id : 10] +(47) ColumnarToRow [codegen id : 4] +Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] + +(48) HashAggregate [codegen id : 4] Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] Aggregate Attributes [1]: [sum#56] Results [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] -(48) Exchange +(49) Exchange Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] -Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(49) HashAggregate [codegen id : 11] +(50) HashAggregate [codegen id : 5] Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58] Results [2]: [c_customer_id#42 AS customer_id#59, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58,18,2) AS year_total#60] -(50) Filter [codegen id : 11] +(51) Filter [codegen id : 5] Input [2]: [customer_id#59, year_total#60] Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.00)) -(51) BroadcastExchange +(52) BroadcastExchange Input [2]: [customer_id#59, year_total#60] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(52) BroadcastHashJoin [codegen id : 16] +(53) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#59] Join type: Inner Join condition: None -(53) Project [codegen id : 16] +(54) Project [codegen id : 8] Output [5]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60] Input [6]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, customer_id#59, year_total#60] -(54) Scan parquet spark_catalog.default.customer +(55) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(55) CometFilter +(56) CometFilter Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] Condition : (isnotnull(c_customer_sk#61) AND isnotnull(c_customer_id#62)) -(56) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] - (57) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Batched: true @@ -357,37 +363,35 @@ ReadSchema: struct 0.00) THEN (year_total#79 / year_total#60) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END) -(71) Project [codegen id : 16] +(71) Project [codegen id : 8] Output [1]: [customer_preferred_cust_flag#39] Input [7]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60, customer_id#78, year_total#79] @@ -425,7 +429,7 @@ Arguments: 100, [customer_preferred_cust_flag#39 ASC NULLS FIRST], [customer_pre ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (76) +- * ColumnarToRow (75) +- CometFilter (74) @@ -448,7 +452,7 @@ Input [2]: [d_date_sk#14, d_year#15] (76) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 BroadcastExchange (80) @@ -473,9 +477,9 @@ Input [2]: [d_date_sk#34, d_year#35] (80) BroadcastExchange Input [2]: [d_date_sk#34, d_year#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#33 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt index 562b5fdf29..fe6b3a625a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_preferred_cust_flag] - WholeStageCodegen (16) + WholeStageCodegen (8) Project [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] @@ -10,114 +10,98 @@ TakeOrderedAndProject [customer_preferred_cust_flag] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + WholeStageCodegen (2) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange #7 CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #13 + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt index 6cf7f4b088..d7d56bf6b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * HashAggregate (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.web_sales @@ -33,118 +35,129 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [ws_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] +Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(15) HashAggregate [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Aggregate Attributes [1]: [sum#13] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] -(14) Exchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +(17) HashAggregate [codegen id : 2] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] -Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#15] +Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#15,17,2) AS itemrevenue#16, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#15,17,2) AS _w0#17, i_item_id#6] -(16) Exchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(18) Exchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +(19) Sort [codegen id : 3] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(20) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] +Arguments: [sum(_w0#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9] -(19) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] -Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] +(21) Project [codegen id : 4] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, ((_w0#17 * 100) / _we0#18) AS revenueratio#19, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6, _we0#18] -(20) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +(22) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#19 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) -(21) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(23) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt index fae1c6dba1..4bdb7ae0f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,12 +29,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt index a647b1f053..e137f931ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt @@ -1,38 +1,37 @@ == Physical Plan == -* HashAggregate (34) -+- Exchange (33) - +- * HashAggregate (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store (4) - : : : +- BroadcastExchange (14) - : : : +- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (17) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.customer_demographics (20) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.household_demographics (26) +* HashAggregate (33) ++- Exchange (32) + +- * HashAggregate (31) + +- * ColumnarToRow (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometProject (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.customer_address (8) + : : +- CometBroadcastExchange (17) + : : +- CometProject (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.date_dim (14) + : +- CometBroadcastExchange (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.customer_demographics (20) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.household_demographics (25) (1) Scan parquet spark_catalog.default.store_sales @@ -47,186 +46,185 @@ ReadSchema: struct= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) -(3) ColumnarToRow [codegen id : 6] -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] - -(4) Scan parquet spark_catalog.default.store +(3) Scan parquet spark_catalog.default.store Output [1]: [s_store_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [1]: [s_store_sk#12] Condition : isnotnull(s_store_sk#12) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [1]: [s_store_sk#12] +Arguments: [s_store_sk#12] -(7) BroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +Right output [1]: [s_store_sk#12] +Arguments: [ss_store_sk#4], [s_store_sk#12], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 6] -Output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +(7) CometProject Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] +Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -(10) Scan parquet spark_catalog.default.customer_address +(8) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] 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, [OH,TX]),In(ca_state, [KY,NM,OR])),In(ca_state, [MS,TX,VA]))] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((ca_state#14 IN (TX,OH) OR ca_state#14 IN (OR,NM,KY)) OR ca_state#14 IN (VA,TX,MS))) -(12) CometProject +(10) CometProject Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] Arguments: [ca_address_sk#13, ca_state#14], [ca_address_sk#13, ca_state#14] -(13) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#13, ca_state#14] - -(14) BroadcastExchange +(11) CometBroadcastExchange Input [2]: [ca_address_sk#13, ca_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [ca_address_sk#13, ca_state#14] -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#13] -Join type: Inner -Join condition: ((((ca_state#14 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#14 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#14 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) +(12) CometBroadcastHashJoin +Left output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +Right output [2]: [ca_address_sk#13, ca_state#14] +Arguments: [ss_addr_sk#3], [ca_address_sk#13], Inner, ((((ca_state#14 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#14 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#14 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))), BuildRight -(16) Project [codegen id : 6] -Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] +(13) CometProject Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#14] +Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -(17) ReusedExchange [Reuses operator id: 39] -Output [1]: [d_date_sk#16] +(14) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_year#17] +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 + +(15) CometFilter +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(16) CometProject +Input [2]: [d_date_sk#16, d_year#17] +Arguments: [d_date_sk#16], [d_date_sk#16] + +(17) CometBroadcastExchange +Input [1]: [d_date_sk#16] +Arguments: [d_date_sk#16] -(19) Project [codegen id : 6] -Output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] +(18) CometBroadcastHashJoin +Left output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] +Right output [1]: [d_date_sk#16] +Arguments: [ss_sold_date_sk#10], [d_date_sk#16], Inner, BuildRight + +(19) CometProject Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#16] +Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] (20) Scan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (21) CometFilter -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] -Condition : (isnotnull(cd_demo_sk#17) AND ((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) OR ((cd_marital_status#18 = S) AND (cd_education_status#19 = College ))) OR ((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )))) - -(22) ColumnarToRow [codegen id : 4] -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] +Condition : (isnotnull(cd_demo_sk#18) AND ((((cd_marital_status#19 = M) AND (cd_education_status#20 = Advanced Degree )) OR ((cd_marital_status#19 = S) AND (cd_education_status#20 = College ))) OR ((cd_marital_status#19 = W) AND (cd_education_status#20 = 2 yr Degree )))) -(23) BroadcastExchange -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(22) CometBroadcastExchange +Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] +Arguments: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#17] -Join type: Inner -Join condition: ((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +(23) CometBroadcastHashJoin +Left output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] +Right output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] +Arguments: [ss_cdemo_sk#1], [cd_demo_sk#18], Inner, ((((((cd_marital_status#19 = M) AND (cd_education_status#20 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#19 = S) AND (cd_education_status#20 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#19 = W) AND (cd_education_status#20 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))), BuildRight -(25) Project [codegen id : 6] -Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19] -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +(24) CometProject +Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] +Arguments: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20], [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20] -(26) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_dep_count#21] +(25) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#21, hd_dep_count#22] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] ReadSchema: struct -(27) CometFilter -Input [2]: [hd_demo_sk#20, hd_dep_count#21] -Condition : (isnotnull(hd_demo_sk#20) AND ((hd_dep_count#21 = 3) OR (hd_dep_count#21 = 1))) +(26) CometFilter +Input [2]: [hd_demo_sk#21, hd_dep_count#22] +Condition : (isnotnull(hd_demo_sk#21) AND ((hd_dep_count#22 = 3) OR (hd_dep_count#22 = 1))) -(28) ColumnarToRow [codegen id : 5] -Input [2]: [hd_demo_sk#20, hd_dep_count#21] +(27) CometBroadcastExchange +Input [2]: [hd_demo_sk#21, hd_dep_count#22] +Arguments: [hd_demo_sk#21, hd_dep_count#22] -(29) BroadcastExchange -Input [2]: [hd_demo_sk#20, hd_dep_count#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(28) CometBroadcastHashJoin +Left output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20] +Right output [2]: [hd_demo_sk#21, hd_dep_count#22] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#21], Inner, (((((((cd_marital_status#19 = M) AND (cd_education_status#20 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#22 = 3)) OR (((((cd_marital_status#19 = S) AND (cd_education_status#20 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#22 = 1))) OR (((((cd_marital_status#19 = W) AND (cd_education_status#20 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#22 = 1))), BuildRight -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: (((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#21 = 3)) OR (((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#21 = 1))) OR (((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#21 = 1))) +(29) CometProject +Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20, hd_demo_sk#21, hd_dep_count#22] +Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -(31) Project [codegen id : 6] -Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19, hd_demo_sk#20, hd_dep_count#21] +(30) ColumnarToRow [codegen id : 1] +Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -(32) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 1] Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Keys: [] Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [7]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28] -Results [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] +Aggregate Attributes [7]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29] +Results [7]: [sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36] -(33) Exchange -Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] +(32) Exchange +Input [7]: [sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(34) HashAggregate [codegen id : 7] -Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] +(33) HashAggregate [codegen id : 2] +Input [7]: [sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [4]: [avg(ss_quantity#5)#36, avg(UnscaledValue(ss_ext_sales_price#7))#37, avg(UnscaledValue(ss_ext_wholesale_cost#8))#38, sum(UnscaledValue(ss_ext_wholesale_cost#8))#39] -Results [4]: [avg(ss_quantity#5)#36 AS avg(ss_quantity)#40, cast((avg(UnscaledValue(ss_ext_sales_price#7))#37 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#41, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#38 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#42, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#39,17,2) AS sum(ss_ext_wholesale_cost)#43] +Aggregate Attributes [4]: [avg(ss_quantity#5)#37, avg(UnscaledValue(ss_ext_sales_price#7))#38, avg(UnscaledValue(ss_ext_wholesale_cost#8))#39, sum(UnscaledValue(ss_ext_wholesale_cost#8))#40] +Results [4]: [avg(ss_quantity#5)#37 AS avg(ss_quantity)#41, cast((avg(UnscaledValue(ss_ext_sales_price#7))#38 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#42, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#39 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#43, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#40,17,2) AS sum(ss_ext_wholesale_cost)#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (39) -+- * ColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan parquet spark_catalog.default.date_dim (35) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(35) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_year#44] +(34) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_year#17] 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 -(36) CometFilter -Input [2]: [d_date_sk#16, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2001)) AND isnotnull(d_date_sk#16)) +(35) CometFilter +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(37) CometProject -Input [2]: [d_date_sk#16, d_year#44] +(36) CometProject +Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(38) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] -(39) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt index 5e5fc41f83..ef54c7add6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt @@ -1,21 +1,21 @@ -WholeStageCodegen (7) +WholeStageCodegen (2) HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] InputAdapter Exchange #1 - WholeStageCodegen (6) + WholeStageCodegen (1) 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_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] - Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_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,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -26,34 +26,20 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #4 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #6 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastExchange #7 + CometFilter [hd_demo_sk,hd_dep_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt index 565cb97da6..dc25c6e683 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt @@ -28,43 +28,43 @@ TakeOrderedAndProject (105) : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) : : : :- * HashAggregate (35) : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) + : : : : +- * ColumnarToRow (33) + : : : : +- CometHashAggregate (32) + : : : : +- CometProject (31) + : : : : +- CometBroadcastHashJoin (30) + : : : : :- CometProject (28) + : : : : : +- CometBroadcastHashJoin (27) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- CometBroadcastExchange (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (10) + : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometProject (23) + : : : : : +- CometBroadcastHashJoin (22) + : : : : : :- CometProject (17) + : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : :- CometFilter (12) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : +- CometBroadcastExchange (15) + : : : : : : +- CometFilter (14) + : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : +- CometBroadcastExchange (21) + : : : : : +- CometProject (20) + : : : : : +- CometFilter (19) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) + : : : : +- ReusedExchange (29) : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * ColumnarToRow (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) + : : : +- * ColumnarToRow (44) + : : : +- CometProject (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (38) + : : : +- ReusedExchange (41) : : +- BroadcastExchange (57) : : +- * BroadcastHashJoin LeftSemi BuildRight (56) : : :- * ColumnarToRow (54) @@ -118,7 +118,7 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 25] +(3) ColumnarToRow [codegen id : 11] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (4) Scan parquet spark_catalog.default.item @@ -132,7 +132,7 @@ ReadSchema: struct Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -172,629 +166,625 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) ColumnarToRow [codegen id : 1] +(15) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(19) BroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(17) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#24] +(18) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(19) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) + +(20) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] + +(21) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(22) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight + +(23) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(24) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(25) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(28) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#25] +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(31) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] (34) Exchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] +Results [3]: [brand_id#27, class_id#28, category_id#29] (36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (37) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(42) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#36] +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] (45) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(48) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(49) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] (50) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [ss_item_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (52) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (53) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : isnotnull(i_item_sk#38) +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : isnotnull(i_item_sk#39) -(54) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] (55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (57) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(58) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(59) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 129] -Output [1]: [d_date_sk#42] +(60) ReusedExchange [Reuses operator id: 127] +Output [1]: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(62) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(63) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] (64) Exchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(65) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(65) HashAggregate [codegen id : 12] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#51, count(1)#50 AS number_sales#52] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#52, count(1)#51 AS number_sales#53] -(66) Filter [codegen id : 26] -Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) +(66) Filter [codegen id : 12] +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(67) Project [codegen id : 26] -Output [6]: [sales#51, number_sales#52, store AS channel#55, i_brand_id#39 AS i_brand_id#56, i_class_id#40 AS i_class_id#57, i_category_id#41 AS i_category_id#58] -Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] +(67) Project [codegen id : 12] +Output [6]: [sales#52, number_sales#53, store AS channel#56, i_brand_id#40 AS i_brand_id#57, i_class_id#41 AS i_class_id#58, i_category_id#42 AS i_category_id#59] +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#52, number_sales#53] (68) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +Output [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#62), dynamicpruningexpression(cs_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(cs_sold_date_sk#63), dynamicpruningexpression(cs_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (69) CometFilter -Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] -Condition : isnotnull(cs_item_sk#59) +Input [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] +Condition : isnotnull(cs_item_sk#60) -(70) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +(70) ColumnarToRow [codegen id : 23] +Input [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] (71) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#64] +Output [1]: [ss_item_sk#65] -(72) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#59] -Right keys [1]: [ss_item_sk#64] +(72) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#60] +Right keys [1]: [ss_item_sk#65] Join type: LeftSemi Join condition: None (73) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#65, i_brand_id#66, i_class_id#67, i_category_id#68] +Output [4]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69] -(74) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#59] -Right keys [1]: [i_item_sk#65] +(74) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#60] +Right keys [1]: [i_item_sk#66] Join type: Inner Join condition: None -(75) Project [codegen id : 51] -Output [6]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#66, i_class_id#67, i_category_id#68] -Input [8]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_item_sk#65, i_brand_id#66, i_class_id#67, i_category_id#68] +(75) Project [codegen id : 23] +Output [6]: [cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69] +Input [8]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69] -(76) ReusedExchange [Reuses operator id: 129] -Output [1]: [d_date_sk#69] +(76) ReusedExchange [Reuses operator id: 127] +Output [1]: [d_date_sk#70] -(77) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#62] -Right keys [1]: [d_date_sk#69] +(77) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_sold_date_sk#63] +Right keys [1]: [d_date_sk#70] Join type: Inner Join condition: None -(78) Project [codegen id : 51] -Output [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#66, i_class_id#67, i_category_id#68] -Input [7]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#66, i_class_id#67, i_category_id#68, d_date_sk#69] +(78) Project [codegen id : 23] +Output [5]: [cs_quantity#61, cs_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] +Input [7]: [cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_brand_id#67, i_class_id#68, i_category_id#69, d_date_sk#70] -(79) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#66, i_class_id#67, i_category_id#68] -Keys [3]: [i_brand_id#66, i_class_id#67, i_category_id#68] -Functions [2]: [partial_sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), partial_count(1)] -Aggregate Attributes [3]: [sum#70, isEmpty#71, count#72] -Results [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] +(79) HashAggregate [codegen id : 23] +Input [5]: [cs_quantity#61, cs_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] +Keys [3]: [i_brand_id#67, i_class_id#68, i_category_id#69] +Functions [2]: [partial_sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62)), partial_count(1)] +Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73] +Results [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#74, isEmpty#75, count#76] (80) Exchange -Input [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] -Arguments: hashpartitioning(i_brand_id#66, i_class_id#67, i_category_id#68, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#74, isEmpty#75, count#76] +Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(81) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] -Keys [3]: [i_brand_id#66, i_class_id#67, i_category_id#68] -Functions [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#76, count(1)#77] -Results [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#76 AS sales#78, count(1)#77 AS number_sales#79] +(81) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum#74, isEmpty#75, count#76] +Keys [3]: [i_brand_id#67, i_class_id#68, i_category_id#69] +Functions [2]: [sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62))#77, count(1)#78] +Results [5]: [i_brand_id#67, i_class_id#68, i_category_id#69, sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62))#77 AS sales#79, count(1)#78 AS number_sales#80] -(82) Filter [codegen id : 52] -Input [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sales#78, number_sales#79] -Condition : (isnotnull(sales#78) AND (cast(sales#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) +(82) Filter [codegen id : 24] +Input [5]: [i_brand_id#67, i_class_id#68, i_category_id#69, sales#79, number_sales#80] +Condition : (isnotnull(sales#79) AND (cast(sales#79 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(83) Project [codegen id : 52] -Output [6]: [sales#78, number_sales#79, catalog AS channel#80, i_brand_id#66, i_class_id#67, i_category_id#68] -Input [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sales#78, number_sales#79] +(83) Project [codegen id : 24] +Output [6]: [sales#79, number_sales#80, catalog AS channel#81, i_brand_id#67, i_class_id#68, i_category_id#69] +Input [5]: [i_brand_id#67, i_class_id#68, i_category_id#69, sales#79, number_sales#80] (84) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] +Output [4]: [ws_item_sk#82, ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#85)] +PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#86)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (85) CometFilter -Input [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] -Condition : isnotnull(ws_item_sk#81) +Input [4]: [ws_item_sk#82, ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85] +Condition : isnotnull(ws_item_sk#82) -(86) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] +(86) ColumnarToRow [codegen id : 35] +Input [4]: [ws_item_sk#82, ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85] (87) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#86] +Output [1]: [ss_item_sk#87] -(88) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#81] -Right keys [1]: [ss_item_sk#86] +(88) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#82] +Right keys [1]: [ss_item_sk#87] Join type: LeftSemi Join condition: None (89) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#87, i_brand_id#88, i_class_id#89, i_category_id#90] +Output [4]: [i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91] -(90) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#81] -Right keys [1]: [i_item_sk#87] +(90) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#82] +Right keys [1]: [i_item_sk#88] Join type: Inner Join condition: None -(91) Project [codegen id : 77] -Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_brand_id#88, i_class_id#89, i_category_id#90] -Input [8]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_item_sk#87, i_brand_id#88, i_class_id#89, i_category_id#90] +(91) Project [codegen id : 35] +Output [6]: [ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85, i_brand_id#89, i_class_id#90, i_category_id#91] +Input [8]: [ws_item_sk#82, ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85, i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91] -(92) ReusedExchange [Reuses operator id: 129] -Output [1]: [d_date_sk#91] +(92) ReusedExchange [Reuses operator id: 127] +Output [1]: [d_date_sk#92] -(93) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#84] -Right keys [1]: [d_date_sk#91] +(93) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_sold_date_sk#85] +Right keys [1]: [d_date_sk#92] Join type: Inner Join condition: None -(94) Project [codegen id : 77] -Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#88, i_class_id#89, i_category_id#90] -Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_brand_id#88, i_class_id#89, i_category_id#90, d_date_sk#91] +(94) Project [codegen id : 35] +Output [5]: [ws_quantity#83, ws_list_price#84, i_brand_id#89, i_class_id#90, i_category_id#91] +Input [7]: [ws_quantity#83, ws_list_price#84, ws_sold_date_sk#85, i_brand_id#89, i_class_id#90, i_category_id#91, d_date_sk#92] -(95) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#88, i_class_id#89, i_category_id#90] -Keys [3]: [i_brand_id#88, i_class_id#89, i_category_id#90] -Functions [2]: [partial_sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83)), partial_count(1)] -Aggregate Attributes [3]: [sum#92, isEmpty#93, count#94] -Results [6]: [i_brand_id#88, i_class_id#89, i_category_id#90, sum#95, isEmpty#96, count#97] +(95) HashAggregate [codegen id : 35] +Input [5]: [ws_quantity#83, ws_list_price#84, i_brand_id#89, i_class_id#90, i_category_id#91] +Keys [3]: [i_brand_id#89, i_class_id#90, i_category_id#91] +Functions [2]: [partial_sum((cast(ws_quantity#83 as decimal(10,0)) * ws_list_price#84)), partial_count(1)] +Aggregate Attributes [3]: [sum#93, isEmpty#94, count#95] +Results [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] (96) Exchange -Input [6]: [i_brand_id#88, i_class_id#89, i_category_id#90, sum#95, isEmpty#96, count#97] -Arguments: hashpartitioning(i_brand_id#88, i_class_id#89, i_category_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Input [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] +Arguments: hashpartitioning(i_brand_id#89, i_class_id#90, i_category_id#91, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(97) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#88, i_class_id#89, i_category_id#90, sum#95, isEmpty#96, count#97] -Keys [3]: [i_brand_id#88, i_class_id#89, i_category_id#90] -Functions [2]: [sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83))#98, count(1)#99] -Results [5]: [i_brand_id#88, i_class_id#89, i_category_id#90, sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83))#98 AS sales#100, count(1)#99 AS number_sales#101] +(97) HashAggregate [codegen id : 36] +Input [6]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum#96, isEmpty#97, count#98] +Keys [3]: [i_brand_id#89, i_class_id#90, i_category_id#91] +Functions [2]: [sum((cast(ws_quantity#83 as decimal(10,0)) * ws_list_price#84)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#83 as decimal(10,0)) * ws_list_price#84))#99, count(1)#100] +Results [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sum((cast(ws_quantity#83 as decimal(10,0)) * ws_list_price#84))#99 AS sales#101, count(1)#100 AS number_sales#102] -(98) Filter [codegen id : 78] -Input [5]: [i_brand_id#88, i_class_id#89, i_category_id#90, sales#100, number_sales#101] -Condition : (isnotnull(sales#100) AND (cast(sales#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) +(98) Filter [codegen id : 36] +Input [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sales#101, number_sales#102] +Condition : (isnotnull(sales#101) AND (cast(sales#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(99) Project [codegen id : 78] -Output [6]: [sales#100, number_sales#101, web AS channel#102, i_brand_id#88, i_class_id#89, i_category_id#90] -Input [5]: [i_brand_id#88, i_class_id#89, i_category_id#90, sales#100, number_sales#101] +(99) Project [codegen id : 36] +Output [6]: [sales#101, number_sales#102, web AS channel#103, i_brand_id#89, i_class_id#90, i_category_id#91] +Input [5]: [i_brand_id#89, i_class_id#90, i_category_id#91, sales#101, number_sales#102] (100) Union -(101) Expand [codegen id : 79] -Input [6]: [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58] -Arguments: [[sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58, 0], [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, null, 1], [sales#51, number_sales#52, channel#55, i_brand_id#56, null, null, 3], [sales#51, number_sales#52, channel#55, null, null, null, 7], [sales#51, number_sales#52, null, null, null, null, 15]], [sales#51, number_sales#52, channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107] +(101) Expand [codegen id : 37] +Input [6]: [sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, i_category_id#59] +Arguments: [[sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, i_category_id#59, 0], [sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, null, 1], [sales#52, number_sales#53, channel#56, i_brand_id#57, null, null, 3], [sales#52, number_sales#53, channel#56, null, null, null, 7], [sales#52, number_sales#53, null, null, null, null, 15]], [sales#52, number_sales#53, channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108] -(102) HashAggregate [codegen id : 79] -Input [7]: [sales#51, number_sales#52, channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107] -Keys [5]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107] -Functions [2]: [partial_sum(sales#51), partial_sum(number_sales#52)] -Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] -Results [8]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107, sum#111, isEmpty#112, sum#113] +(102) HashAggregate [codegen id : 37] +Input [7]: [sales#52, number_sales#53, channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108] +Keys [5]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108] +Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] +Aggregate Attributes [3]: [sum#109, isEmpty#110, sum#111] +Results [8]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, sum#112, isEmpty#113, sum#114] (103) Exchange -Input [8]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107, sum#111, isEmpty#112, sum#113] -Arguments: hashpartitioning(channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [8]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, sum#112, isEmpty#113, sum#114] +Arguments: hashpartitioning(channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(104) HashAggregate [codegen id : 80] -Input [8]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107, sum#111, isEmpty#112, sum#113] -Keys [5]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107] -Functions [2]: [sum(sales#51), sum(number_sales#52)] -Aggregate Attributes [2]: [sum(sales#51)#114, sum(number_sales#52)#115] -Results [6]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, sum(sales#51)#114 AS sum(sales)#116, sum(number_sales#52)#115 AS sum(number_sales)#117] +(104) HashAggregate [codegen id : 38] +Input [8]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108, sum#112, isEmpty#113, sum#114] +Keys [5]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, spark_grouping_id#108] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#115, sum(number_sales#53)#116] +Results [6]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, sum(sales#52)#115 AS sum(sales)#117, sum(number_sales#53)#116 AS sum(number_sales)#118] (105) TakeOrderedAndProject -Input [6]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, sum(sales)#116, sum(number_sales)#117] -Arguments: 100, [channel#103 ASC NULLS FIRST, i_brand_id#104 ASC NULLS FIRST, i_class_id#105 ASC NULLS FIRST, i_category_id#106 ASC NULLS FIRST], [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, sum(sales)#116, sum(number_sales)#117] +Input [6]: [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, sum(sales)#117, sum(number_sales)#118] +Arguments: 100, [channel#104 ASC NULLS FIRST, i_brand_id#105 ASC NULLS FIRST, i_class_id#106 ASC NULLS FIRST, i_category_id#107 ASC NULLS FIRST], [channel#104, i_brand_id#105, i_class_id#106, i_category_id#107, sum(sales)#117, sum(number_sales)#118] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* HashAggregate (124) -+- Exchange (123) - +- * HashAggregate (122) - +- Union (121) - :- * Project (110) - : +- * BroadcastHashJoin Inner BuildRight (109) - : :- * ColumnarToRow (107) - : : +- CometScan parquet spark_catalog.default.store_sales (106) - : +- ReusedExchange (108) - :- * Project (115) - : +- * BroadcastHashJoin Inner BuildRight (114) - : :- * ColumnarToRow (112) - : : +- CometScan parquet spark_catalog.default.catalog_sales (111) - : +- ReusedExchange (113) - +- * Project (120) - +- * BroadcastHashJoin Inner BuildRight (119) - :- * ColumnarToRow (117) - : +- CometScan parquet spark_catalog.default.web_sales (116) - +- ReusedExchange (118) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (122) ++- Exchange (121) + +- * HashAggregate (120) + +- * ColumnarToRow (119) + +- CometUnion (118) + :- CometProject (109) + : +- CometBroadcastHashJoin (108) + : :- CometScan parquet spark_catalog.default.store_sales (106) + : +- ReusedExchange (107) + :- CometProject (113) + : +- CometBroadcastHashJoin (112) + : :- CometScan parquet spark_catalog.default.catalog_sales (110) + : +- ReusedExchange (111) + +- CometProject (117) + +- CometBroadcastHashJoin (116) + :- CometScan parquet spark_catalog.default.web_sales (114) + +- ReusedExchange (115) (106) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#118, ss_list_price#119, ss_sold_date_sk#120] +Output [3]: [ss_quantity#119, ss_list_price#120, ss_sold_date_sk#121] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#120), dynamicpruningexpression(ss_sold_date_sk#120 IN dynamicpruning#121)] +PartitionFilters: [isnotnull(ss_sold_date_sk#121), dynamicpruningexpression(ss_sold_date_sk#121 IN dynamicpruning#122)] ReadSchema: struct -(107) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#118, ss_list_price#119, ss_sold_date_sk#120] - -(108) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#122] +(107) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#123] -(109) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#120] -Right keys [1]: [d_date_sk#122] -Join type: Inner -Join condition: None +(108) CometBroadcastHashJoin +Left output [3]: [ss_quantity#119, ss_list_price#120, ss_sold_date_sk#121] +Right output [1]: [d_date_sk#123] +Arguments: [ss_sold_date_sk#121], [d_date_sk#123], Inner, BuildRight -(110) Project [codegen id : 2] -Output [2]: [ss_quantity#118 AS quantity#123, ss_list_price#119 AS list_price#124] -Input [4]: [ss_quantity#118, ss_list_price#119, ss_sold_date_sk#120, d_date_sk#122] +(109) CometProject +Input [4]: [ss_quantity#119, ss_list_price#120, ss_sold_date_sk#121, d_date_sk#123] +Arguments: [quantity#124, list_price#125], [ss_quantity#119 AS quantity#124, ss_list_price#120 AS list_price#125] -(111) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#125, cs_list_price#126, cs_sold_date_sk#127] +(110) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#126, cs_list_price#127, cs_sold_date_sk#128] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#127), dynamicpruningexpression(cs_sold_date_sk#127 IN dynamicpruning#128)] +PartitionFilters: [isnotnull(cs_sold_date_sk#128), dynamicpruningexpression(cs_sold_date_sk#128 IN dynamicpruning#129)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#125, cs_list_price#126, cs_sold_date_sk#127] +(111) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#130] -(113) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#129] +(112) CometBroadcastHashJoin +Left output [3]: [cs_quantity#126, cs_list_price#127, cs_sold_date_sk#128] +Right output [1]: [d_date_sk#130] +Arguments: [cs_sold_date_sk#128], [d_date_sk#130], Inner, BuildRight -(114) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#127] -Right keys [1]: [d_date_sk#129] -Join type: Inner -Join condition: None - -(115) Project [codegen id : 4] -Output [2]: [cs_quantity#125 AS quantity#130, cs_list_price#126 AS list_price#131] -Input [4]: [cs_quantity#125, cs_list_price#126, cs_sold_date_sk#127, d_date_sk#129] +(113) CometProject +Input [4]: [cs_quantity#126, cs_list_price#127, cs_sold_date_sk#128, d_date_sk#130] +Arguments: [quantity#131, list_price#132], [cs_quantity#126 AS quantity#131, cs_list_price#127 AS list_price#132] -(116) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#132, ws_list_price#133, ws_sold_date_sk#134] +(114) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#133, ws_list_price#134, ws_sold_date_sk#135] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#134), dynamicpruningexpression(ws_sold_date_sk#134 IN dynamicpruning#135)] +PartitionFilters: [isnotnull(ws_sold_date_sk#135), dynamicpruningexpression(ws_sold_date_sk#135 IN dynamicpruning#136)] ReadSchema: struct -(117) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#132, ws_list_price#133, ws_sold_date_sk#134] +(115) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#137] -(118) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#136] +(116) CometBroadcastHashJoin +Left output [3]: [ws_quantity#133, ws_list_price#134, ws_sold_date_sk#135] +Right output [1]: [d_date_sk#137] +Arguments: [ws_sold_date_sk#135], [d_date_sk#137], Inner, BuildRight -(119) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#134] -Right keys [1]: [d_date_sk#136] -Join type: Inner -Join condition: None +(117) CometProject +Input [4]: [ws_quantity#133, ws_list_price#134, ws_sold_date_sk#135, d_date_sk#137] +Arguments: [quantity#138, list_price#139], [ws_quantity#133 AS quantity#138, ws_list_price#134 AS list_price#139] -(120) Project [codegen id : 6] -Output [2]: [ws_quantity#132 AS quantity#137, ws_list_price#133 AS list_price#138] -Input [4]: [ws_quantity#132, ws_list_price#133, ws_sold_date_sk#134, d_date_sk#136] +(118) CometUnion +Child 0 Input [2]: [quantity#124, list_price#125] +Child 1 Input [2]: [quantity#131, list_price#132] +Child 2 Input [2]: [quantity#138, list_price#139] -(121) Union +(119) ColumnarToRow [codegen id : 1] +Input [2]: [quantity#124, list_price#125] -(122) HashAggregate [codegen id : 7] -Input [2]: [quantity#123, list_price#124] +(120) HashAggregate [codegen id : 1] +Input [2]: [quantity#124, list_price#125] Keys: [] -Functions [1]: [partial_avg((cast(quantity#123 as decimal(10,0)) * list_price#124))] -Aggregate Attributes [2]: [sum#139, count#140] -Results [2]: [sum#141, count#142] +Functions [1]: [partial_avg((cast(quantity#124 as decimal(10,0)) * list_price#125))] +Aggregate Attributes [2]: [sum#140, count#141] +Results [2]: [sum#142, count#143] -(123) Exchange -Input [2]: [sum#141, count#142] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(121) Exchange +Input [2]: [sum#142, count#143] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(124) HashAggregate [codegen id : 8] -Input [2]: [sum#141, count#142] +(122) HashAggregate [codegen id : 2] +Input [2]: [sum#142, count#143] Keys: [] -Functions [1]: [avg((cast(quantity#123 as decimal(10,0)) * list_price#124))] -Aggregate Attributes [1]: [avg((cast(quantity#123 as decimal(10,0)) * list_price#124))#143] -Results [1]: [avg((cast(quantity#123 as decimal(10,0)) * list_price#124))#143 AS average_sales#144] +Functions [1]: [avg((cast(quantity#124 as decimal(10,0)) * list_price#125))] +Aggregate Attributes [1]: [avg((cast(quantity#124 as decimal(10,0)) * list_price#125))#144] +Results [1]: [avg((cast(quantity#124 as decimal(10,0)) * list_price#125))#144 AS average_sales#145] -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#120 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#121 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 111 Hosting Expression = cs_sold_date_sk#127 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#128 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 116 Hosting Expression = ws_sold_date_sk#134 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 114 Hosting Expression = ws_sold_date_sk#135 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (129) -+- * ColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan parquet spark_catalog.default.date_dim (125) +BroadcastExchange (127) ++- * ColumnarToRow (126) + +- CometProject (125) + +- CometFilter (124) + +- CometScan parquet spark_catalog.default.date_dim (123) -(125) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#42, d_year#145, d_moy#146] +(123) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#146, d_moy#147] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(126) CometFilter -Input [3]: [d_date_sk#42, d_year#145, d_moy#146] -Condition : ((((isnotnull(d_year#145) AND isnotnull(d_moy#146)) AND (d_year#145 = 2001)) AND (d_moy#146 = 11)) AND isnotnull(d_date_sk#42)) +(124) CometFilter +Input [3]: [d_date_sk#43, d_year#146, d_moy#147] +Condition : ((((isnotnull(d_year#146) AND isnotnull(d_moy#147)) AND (d_year#146 = 2001)) AND (d_moy#147 = 11)) AND isnotnull(d_date_sk#43)) -(127) CometProject -Input [3]: [d_date_sk#42, d_year#145, d_moy#146] -Arguments: [d_date_sk#42], [d_date_sk#42] +(125) CometProject +Input [3]: [d_date_sk#43, d_year#146, d_moy#147] +Arguments: [d_date_sk#43], [d_date_sk#43] -(128) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(126) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(129) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(127) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (134) -+- * ColumnarToRow (133) - +- CometProject (132) - +- CometFilter (131) - +- CometScan parquet spark_catalog.default.date_dim (130) +BroadcastExchange (132) ++- * ColumnarToRow (131) + +- CometProject (130) + +- CometFilter (129) + +- CometScan parquet spark_catalog.default.date_dim (128) -(130) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#147] +(128) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#148] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(131) CometFilter -Input [2]: [d_date_sk#25, d_year#147] -Condition : (((isnotnull(d_year#147) AND (d_year#147 >= 1999)) AND (d_year#147 <= 2001)) AND isnotnull(d_date_sk#25)) +(129) CometFilter +Input [2]: [d_date_sk#26, d_year#148] +Condition : (((isnotnull(d_year#148) AND (d_year#148 >= 1999)) AND (d_year#148 <= 2001)) AND isnotnull(d_date_sk#26)) -(132) CometProject -Input [2]: [d_date_sk#25, d_year#147] -Arguments: [d_date_sk#25], [d_date_sk#25] +(130) CometProject +Input [2]: [d_date_sk#26, d_year#148] +Arguments: [d_date_sk#26], [d_date_sk#26] -(133) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(131) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(134) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +(132) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#63 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt index cf688c4486..72aa4a054e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt @@ -1,56 +1,45 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (80) + WholeStageCodegen (38) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (79) + WholeStageCodegen (37) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (26) + WholeStageCodegen (12) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #3 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #13 - WholeStageCodegen (7) + Exchange #14 + WholeStageCodegen (1) HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 + ColumnarToRow + InputAdapter + CometUnion + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (25) + WholeStageCodegen (11) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -71,7 +60,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (11) + WholeStageCodegen (4) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow @@ -80,19 +69,19 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (10) + WholeStageCodegen (3) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 @@ -103,56 +92,43 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - ReusedExchange [d_date_sk] #7 + CometBroadcastExchange #8 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #9 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange #10 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #11 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #11 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #12 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - InputAdapter - ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + ReusedExchange [d_date_sk] #11 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (23) + BroadcastExchange #13 + WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter @@ -162,14 +138,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [ss_item_sk] #4 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (52) + WholeStageCodegen (24) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (51) + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (23) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -184,17 +160,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (78) + WholeStageCodegen (36) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (77) + Exchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (35) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -209,6 +185,6 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt index 265909ec28..842435d197 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt @@ -23,43 +23,43 @@ TakeOrderedAndProject (84) : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) : : : :- * HashAggregate (35) : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) + : : : : +- * ColumnarToRow (33) + : : : : +- CometHashAggregate (32) + : : : : +- CometProject (31) + : : : : +- CometBroadcastHashJoin (30) + : : : : :- CometProject (28) + : : : : : +- CometBroadcastHashJoin (27) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- CometBroadcastExchange (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (10) + : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometProject (23) + : : : : : +- CometBroadcastHashJoin (22) + : : : : : :- CometProject (17) + : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : :- CometFilter (12) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : +- CometBroadcastExchange (15) + : : : : : : +- CometFilter (14) + : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : +- CometBroadcastExchange (21) + : : : : : +- CometProject (20) + : : : : : +- CometFilter (19) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) + : : : : +- ReusedExchange (29) : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * ColumnarToRow (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) + : : : +- * ColumnarToRow (44) + : : : +- CometProject (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (38) + : : : +- ReusedExchange (41) : : +- BroadcastExchange (57) : : +- * BroadcastHashJoin LeftSemi BuildRight (56) : : :- * ColumnarToRow (54) @@ -97,7 +97,7 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 25] +(3) ColumnarToRow [codegen id : 11] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (4) Scan parquet spark_catalog.default.item @@ -111,7 +111,7 @@ ReadSchema: struct Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -151,609 +145,605 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) ColumnarToRow [codegen id : 1] +(15) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(19) BroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(17) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#24] +(18) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(19) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) + +(20) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] + +(21) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(22) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight + +(23) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(24) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(25) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(28) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#25] +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(31) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] (34) Exchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] +Results [3]: [brand_id#27, class_id#28, category_id#29] (36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (37) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(42) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#36] +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] (45) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(48) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(49) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] (50) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [ss_item_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (52) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (53) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) -(54) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] (55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (57) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(58) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(59) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#42] +(60) ReusedExchange [Reuses operator id: 106] +Output [1]: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(62) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(63) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] (64) Exchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(65) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(65) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [6]: [store AS channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#53, count(1)#51 AS number_sales#54] -(66) Filter [codegen id : 52] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(66) Filter [codegen id : 24] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Condition : (isnotnull(sales#53) AND (cast(sales#53 as decimal(32,6)) > cast(Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (67) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (68) CometFilter -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] -Condition : isnotnull(ss_item_sk#56) +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +Condition : isnotnull(ss_item_sk#57) -(69) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +(69) ColumnarToRow [codegen id : 22] +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] (70) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#61] +Output [1]: [ss_item_sk#62] -(71) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [ss_item_sk#61] +(71) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [ss_item_sk#62] Join type: LeftSemi Join condition: None (72) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] +Output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [i_item_sk#62] +(73) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [i_item_sk#63] Join type: Inner Join condition: None -(74) Project [codegen id : 50] -Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] +(74) Project [codegen id : 22] +Output [6]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] +Input [8]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -(75) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#66] +(75) ReusedExchange [Reuses operator id: 120] +Output [1]: [d_date_sk#67] -(76) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#66] +(76) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_sold_date_sk#60] +Right keys [1]: [d_date_sk#67] Join type: Inner Join condition: None -(77) Project [codegen id : 50] -Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] +(77) Project [codegen id : 22] +Output [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] +Input [7]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66, d_date_sk#67] -(78) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +(78) HashAggregate [codegen id : 22] +Input [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] +Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] +Functions [2]: [partial_sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), partial_count(1)] +Aggregate Attributes [3]: [sum#68, isEmpty#69, count#70] +Results [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73] (79) Exchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73] +Arguments: hashpartitioning(i_brand_id#64, i_class_id#65, i_category_id#66, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(80) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73, count(1)#74] -Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] +(80) HashAggregate [codegen id : 23] +Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73] +Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] +Functions [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#74, count(1)#75] +Results [6]: [store AS channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#74 AS sales#77, count(1)#75 AS number_sales#78] -(81) Filter [codegen id : 51] -Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(81) Filter [codegen id : 23] +Input [6]: [channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78] +Condition : (isnotnull(sales#77) AND (cast(sales#77 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (82) BroadcastExchange -Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] +Input [6]: [channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=8] -(83) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +(83) BroadcastHashJoin [codegen id : 24] +Left keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +Right keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] Join type: Inner Join condition: None (84) TakeOrderedAndProject -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Input [12]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78] +Arguments: 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (103) -+- Exchange (102) - +- * HashAggregate (101) - +- Union (100) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * ColumnarToRow (86) - : : +- CometScan parquet spark_catalog.default.store_sales (85) - : +- ReusedExchange (87) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * ColumnarToRow (91) - : : +- CometScan parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (92) - +- * Project (99) - +- * BroadcastHashJoin Inner BuildRight (98) - :- * ColumnarToRow (96) - : +- CometScan parquet spark_catalog.default.web_sales (95) - +- ReusedExchange (97) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#55, [id=#56] +* HashAggregate (101) ++- Exchange (100) + +- * HashAggregate (99) + +- * ColumnarToRow (98) + +- CometUnion (97) + :- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometScan parquet spark_catalog.default.store_sales (85) + : +- ReusedExchange (86) + :- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometScan parquet spark_catalog.default.catalog_sales (89) + : +- ReusedExchange (90) + +- CometProject (96) + +- CometBroadcastHashJoin (95) + :- CometScan parquet spark_catalog.default.web_sales (93) + +- ReusedExchange (94) (85) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] +Output [3]: [ss_quantity#79, ss_list_price#80, ss_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] +PartitionFilters: [isnotnull(ss_sold_date_sk#81), dynamicpruningexpression(ss_sold_date_sk#81 IN dynamicpruning#82)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] - -(87) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#82] +(86) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#83] -(88) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#80] -Right keys [1]: [d_date_sk#82] -Join type: Inner -Join condition: None +(87) CometBroadcastHashJoin +Left output [3]: [ss_quantity#79, ss_list_price#80, ss_sold_date_sk#81] +Right output [1]: [d_date_sk#83] +Arguments: [ss_sold_date_sk#81], [d_date_sk#83], Inner, BuildRight -(89) Project [codegen id : 2] -Output [2]: [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] -Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] +(88) CometProject +Input [4]: [ss_quantity#79, ss_list_price#80, ss_sold_date_sk#81, d_date_sk#83] +Arguments: [quantity#84, list_price#85], [ss_quantity#79 AS quantity#84, ss_list_price#80 AS list_price#85] -(90) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +(89) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] +PartitionFilters: [isnotnull(cs_sold_date_sk#88), dynamicpruningexpression(cs_sold_date_sk#88 IN dynamicpruning#89)] ReadSchema: struct -(91) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +(90) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#90] -(92) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#89] +(91) CometBroadcastHashJoin +Left output [3]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88] +Right output [1]: [d_date_sk#90] +Arguments: [cs_sold_date_sk#88], [d_date_sk#90], Inner, BuildRight -(93) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#87] -Right keys [1]: [d_date_sk#89] -Join type: Inner -Join condition: None - -(94) Project [codegen id : 4] -Output [2]: [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] -Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] +(92) CometProject +Input [4]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88, d_date_sk#90] +Arguments: [quantity#91, list_price#92], [cs_quantity#86 AS quantity#91, cs_list_price#87 AS list_price#92] -(95) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(93) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] +PartitionFilters: [isnotnull(ws_sold_date_sk#95), dynamicpruningexpression(ws_sold_date_sk#95 IN dynamicpruning#96)] ReadSchema: struct -(96) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(94) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#97] -(97) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#96] +(95) CometBroadcastHashJoin +Left output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95] +Right output [1]: [d_date_sk#97] +Arguments: [ws_sold_date_sk#95], [d_date_sk#97], Inner, BuildRight -(98) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#94] -Right keys [1]: [d_date_sk#96] -Join type: Inner -Join condition: None +(96) CometProject +Input [4]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95, d_date_sk#97] +Arguments: [quantity#98, list_price#99], [ws_quantity#93 AS quantity#98, ws_list_price#94 AS list_price#99] -(99) Project [codegen id : 6] -Output [2]: [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] -Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] +(97) CometUnion +Child 0 Input [2]: [quantity#84, list_price#85] +Child 1 Input [2]: [quantity#91, list_price#92] +Child 2 Input [2]: [quantity#98, list_price#99] -(100) Union +(98) ColumnarToRow [codegen id : 1] +Input [2]: [quantity#84, list_price#85] -(101) HashAggregate [codegen id : 7] -Input [2]: [quantity#83, list_price#84] +(99) HashAggregate [codegen id : 1] +Input [2]: [quantity#84, list_price#85] Keys: [] -Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] -Aggregate Attributes [2]: [sum#99, count#100] -Results [2]: [sum#101, count#102] +Functions [1]: [partial_avg((cast(quantity#84 as decimal(10,0)) * list_price#85))] +Aggregate Attributes [2]: [sum#100, count#101] +Results [2]: [sum#102, count#103] -(102) Exchange -Input [2]: [sum#101, count#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(100) Exchange +Input [2]: [sum#102, count#103] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(103) HashAggregate [codegen id : 8] -Input [2]: [sum#101, count#102] +(101) HashAggregate [codegen id : 2] +Input [2]: [sum#102, count#103] Keys: [] -Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] -Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103] -Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103 AS average_sales#104] +Functions [1]: [avg((cast(quantity#84 as decimal(10,0)) * list_price#85))] +Aggregate Attributes [1]: [avg((cast(quantity#84 as decimal(10,0)) * list_price#85))#104] +Results [1]: [avg((cast(quantity#84 as decimal(10,0)) * list_price#85))#104 AS average_sales#105] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#81 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#88 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#95 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * ColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan parquet spark_catalog.default.date_dim (104) +BroadcastExchange (106) ++- * ColumnarToRow (105) + +- CometProject (104) + +- CometFilter (103) + +- CometScan parquet spark_catalog.default.date_dim (102) -(104) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#42, d_week_seq#105] +(102) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#106), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#107), IsNotNull(d_date_sk)] ReadSchema: struct -(105) CometFilter -Input [2]: [d_date_sk#42, d_week_seq#105] -Condition : ((isnotnull(d_week_seq#105) AND (d_week_seq#105 = ReusedSubquery Subquery scalar-subquery#106, [id=#107])) AND isnotnull(d_date_sk#42)) +(103) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#106] +Condition : ((isnotnull(d_week_seq#106) AND (d_week_seq#106 = ReusedSubquery Subquery scalar-subquery#107, [id=#108])) AND isnotnull(d_date_sk#43)) -(106) CometProject -Input [2]: [d_date_sk#42, d_week_seq#105] -Arguments: [d_date_sk#42], [d_date_sk#42] +(104) CometProject +Input [2]: [d_date_sk#43, d_week_seq#106] +Arguments: [d_date_sk#43], [d_date_sk#43] -(107) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(105) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(108) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(106) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:6 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#106, [id=#107] +Subquery:6 Hosting operator id = 103 Hosting Expression = ReusedSubquery Subquery scalar-subquery#107, [id=#108] -Subquery:7 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#106, [id=#107] -* ColumnarToRow (112) -+- CometProject (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +Subquery:7 Hosting operator id = 102 Hosting Expression = Subquery scalar-subquery#107, [id=#108] +* ColumnarToRow (110) ++- CometProject (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(109) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +(107) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#109, d_year#110, d_moy#111, d_dom#112] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(110) CometFilter -Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] -Condition : (((((isnotnull(d_year#109) AND isnotnull(d_moy#110)) AND isnotnull(d_dom#111)) AND (d_year#109 = 2000)) AND (d_moy#110 = 12)) AND (d_dom#111 = 11)) +(108) CometFilter +Input [4]: [d_week_seq#109, d_year#110, d_moy#111, d_dom#112] +Condition : (((((isnotnull(d_year#110) AND isnotnull(d_moy#111)) AND isnotnull(d_dom#112)) AND (d_year#110 = 2000)) AND (d_moy#111 = 12)) AND (d_dom#112 = 11)) -(111) CometProject -Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] -Arguments: [d_week_seq#108], [d_week_seq#108] +(109) CometProject +Input [4]: [d_week_seq#109, d_year#110, d_moy#111, d_dom#112] +Arguments: [d_week_seq#109], [d_week_seq#109] -(112) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#108] +(110) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#109] Subquery:8 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (115) ++- * ColumnarToRow (114) + +- CometProject (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) -(113) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#112] +(111) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter -Input [2]: [d_date_sk#25, d_year#112] -Condition : (((isnotnull(d_year#112) AND (d_year#112 >= 1999)) AND (d_year#112 <= 2001)) AND isnotnull(d_date_sk#25)) +(112) CometFilter +Input [2]: [d_date_sk#26, d_year#113] +Condition : (((isnotnull(d_year#113) AND (d_year#113 >= 1999)) AND (d_year#113 <= 2001)) AND isnotnull(d_date_sk#26)) -(115) CometProject -Input [2]: [d_date_sk#25, d_year#112] -Arguments: [d_date_sk#25], [d_date_sk#25] +(113) CometProject +Input [2]: [d_date_sk#26, d_year#113] +Arguments: [d_date_sk#26], [d_date_sk#26] -(116) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(114) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(117) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(115) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:9 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:10 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:12 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 -BroadcastExchange (122) -+- * ColumnarToRow (121) - +- CometProject (120) - +- CometFilter (119) - +- CometScan parquet spark_catalog.default.date_dim (118) +Subquery:12 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#60 IN dynamicpruning#61 +BroadcastExchange (120) ++- * ColumnarToRow (119) + +- CometProject (118) + +- CometFilter (117) + +- CometScan parquet spark_catalog.default.date_dim (116) -(118) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_week_seq#113] +(116) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#67, d_week_seq#114] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#114), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#115), IsNotNull(d_date_sk)] ReadSchema: struct -(119) CometFilter -Input [2]: [d_date_sk#66, d_week_seq#113] -Condition : ((isnotnull(d_week_seq#113) AND (d_week_seq#113 = ReusedSubquery Subquery scalar-subquery#114, [id=#115])) AND isnotnull(d_date_sk#66)) +(117) CometFilter +Input [2]: [d_date_sk#67, d_week_seq#114] +Condition : ((isnotnull(d_week_seq#114) AND (d_week_seq#114 = ReusedSubquery Subquery scalar-subquery#115, [id=#116])) AND isnotnull(d_date_sk#67)) -(120) CometProject -Input [2]: [d_date_sk#66, d_week_seq#113] -Arguments: [d_date_sk#66], [d_date_sk#66] +(118) CometProject +Input [2]: [d_date_sk#67, d_week_seq#114] +Arguments: [d_date_sk#67], [d_date_sk#67] -(121) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#66] +(119) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#67] -(122) BroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +(120) BroadcastExchange +Input [1]: [d_date_sk#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:13 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#114, [id=#115] +Subquery:13 Hosting operator id = 117 Hosting Expression = ReusedSubquery Subquery scalar-subquery#115, [id=#116] -Subquery:14 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#114, [id=#115] -* ColumnarToRow (126) -+- CometProject (125) - +- CometFilter (124) - +- CometScan parquet spark_catalog.default.date_dim (123) +Subquery:14 Hosting operator id = 116 Hosting Expression = Subquery scalar-subquery#115, [id=#116] +* ColumnarToRow (124) ++- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) -(123) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +(121) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(124) CometFilter -Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#117) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#117 = 1999)) AND (d_moy#118 = 12)) AND (d_dom#119 = 11)) +(122) CometFilter +Input [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] +Condition : (((((isnotnull(d_year#118) AND isnotnull(d_moy#119)) AND isnotnull(d_dom#120)) AND (d_year#118 = 1999)) AND (d_moy#119 = 12)) AND (d_dom#120 = 11)) -(125) CometProject -Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] -Arguments: [d_week_seq#116], [d_week_seq#116] +(123) CometProject +Input [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] +Arguments: [d_week_seq#117], [d_week_seq#117] -(126) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#116] +(124) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#117] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt index 55aa823ab8..73badab8db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt @@ -1,47 +1,36 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (24) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #12 - WholeStageCodegen (7) + Exchange #13 + WholeStageCodegen (1) HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 + ColumnarToRow + InputAdapter + CometUnion + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (11) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -70,7 +59,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter BroadcastExchange #3 - WholeStageCodegen (11) + WholeStageCodegen (4) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow @@ -79,19 +68,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (10) + WholeStageCodegen (3) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 @@ -102,56 +91,43 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 + CometBroadcastExchange #7 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange #9 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #10 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #11 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) + BroadcastExchange #12 + WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter @@ -162,14 +138,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) + BroadcastExchange #14 + WholeStageCodegen (23) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (22) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -181,7 +157,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 + BroadcastExchange #16 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -199,6 +175,6 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter - ReusedExchange [d_date_sk] #15 + ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt index 13a621f773..a22a52dc68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt @@ -1,26 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.customer (4) - : +- BroadcastExchange (13) - : +- * ColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.customer_address (10) - +- ReusedExchange (16) +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * ColumnarToRow (19) + +- CometProject (18) + +- CometBroadcastHashJoin (17) + :- CometProject (12) + : +- CometBroadcastHashJoin (11) + : :- CometProject (7) + : : +- CometBroadcastHashJoin (6) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : +- CometBroadcastExchange (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.customer (3) + : +- CometBroadcastExchange (10) + : +- CometFilter (9) + : +- CometScan parquet spark_catalog.default.customer_address (8) + +- CometBroadcastExchange (16) + +- CometProject (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.date_dim (13) (1) Scan parquet spark_catalog.default.catalog_sales @@ -35,130 +36,137 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_bill_customer_sk#1) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.customer +(3) Scan parquet spark_catalog.default.customer Output [2]: [c_customer_sk#5, c_current_addr_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [c_customer_sk#5, c_current_addr_sk#6] Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [2]: [c_customer_sk#5, c_current_addr_sk#6] +Arguments: [c_customer_sk#5, c_current_addr_sk#6] -(7) BroadcastExchange -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] +Right output [2]: [c_customer_sk#5, c_current_addr_sk#6] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] +(7) CometProject Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] +Arguments: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6], [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] -(10) Scan parquet spark_catalog.default.customer_address +(8) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] Condition : isnotnull(ca_address_sk#7) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] +Arguments: [ca_address_sk#7, ca_state#8, ca_zip#9] -(13) BroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] +Right output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] +Arguments: [c_current_addr_sk#6], [ca_address_sk#7], Inner, ((substr(ca_zip#9, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#8 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)), BuildRight -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_current_addr_sk#6] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: ((substr(ca_zip#9, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#8 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) - -(15) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9] +(12) CometProject Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#8, ca_zip#9] +Arguments: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9], [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9] -(16) ReusedExchange [Reuses operator id: 27] -Output [1]: [d_date_sk#10] +(13) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_qoy#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(14) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] +Condition : ((((isnotnull(d_qoy#12) AND isnotnull(d_year#11)) AND (d_qoy#12 = 2)) AND (d_year#11 = 2001)) AND isnotnull(d_date_sk#10)) + +(15) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] +Arguments: [d_date_sk#10], [d_date_sk#10] -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(16) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] + +(17) CometBroadcastHashJoin +Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9] +Right output [1]: [d_date_sk#10] +Arguments: [cs_sold_date_sk#3], [d_date_sk#10], Inner, BuildRight -(18) Project [codegen id : 4] -Output [2]: [cs_sales_price#2, ca_zip#9] +(18) CometProject Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9, d_date_sk#10] +Arguments: [cs_sales_price#2, ca_zip#9], [cs_sales_price#2, ca_zip#9] + +(19) ColumnarToRow [codegen id : 1] +Input [2]: [cs_sales_price#2, ca_zip#9] -(19) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [2]: [cs_sales_price#2, ca_zip#9] Keys [1]: [ca_zip#9] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum#11] -Results [2]: [ca_zip#9, sum#12] +Aggregate Attributes [1]: [sum#13] +Results [2]: [ca_zip#9, sum#14] -(20) Exchange -Input [2]: [ca_zip#9, sum#12] -Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) Exchange +Input [2]: [ca_zip#9, sum#14] +Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#9, sum#12] +(22) HashAggregate [codegen id : 2] +Input [2]: [ca_zip#9, sum#14] Keys [1]: [ca_zip#9] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#13] -Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#13,17,2) AS sum(cs_sales_price)#14] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#15] +Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#15,17,2) AS sum(cs_sales_price)#16] -(22) TakeOrderedAndProject -Input [2]: [ca_zip#9, sum(cs_sales_price)#14] -Arguments: 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#14] +(23) TakeOrderedAndProject +Input [2]: [ca_zip#9, sum(cs_sales_price)#16] +Arguments: 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) -(23) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#15, d_qoy#16] +(24) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_qoy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] -Condition : ((((isnotnull(d_qoy#16) AND isnotnull(d_year#15)) AND (d_qoy#16 = 2)) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#10)) +(25) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] +Condition : ((((isnotnull(d_qoy#12) AND isnotnull(d_year#11)) AND (d_qoy#12 = 2)) AND (d_year#11 = 2001)) AND isnotnull(d_date_sk#10)) -(25) CometProject -Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] +(26) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(26) ColumnarToRow [codegen id : 1] +(27) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(27) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt index 5c750b2db8..b0fd9ebf1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] InputAdapter Exchange [ca_zip] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [ca_zip,cs_sales_price] [sum,sum] - Project [cs_sales_price,ca_zip] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] - Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_sales_price,ca_zip] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -23,19 +23,13 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] CometProject [d_date_sk] CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #3 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #4 + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt index 0de98cfb0f..d3739ed757 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt @@ -1,44 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * ColumnarToRow (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometProject (18) + : : : : +- CometBroadcastHashJoin (17) + : : : : :- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (8) + : : : : +- CometBroadcastExchange (16) + : : : : +- CometProject (15) + : : : : +- CometFilter (14) + : : : : +- CometScan parquet spark_catalog.default.date_dim (13) + : : : +- CometBroadcastExchange (22) + : : : +- CometProject (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.date_dim (19) + : : +- ReusedExchange (25) + : +- CometBroadcastExchange (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.store (28) + +- CometBroadcastExchange (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.item (33) (1) Scan parquet spark_catalog.default.store_sales @@ -53,10 +55,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) -(6) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] +Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] +Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] +(7) CometProject Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -(10) Scan parquet spark_catalog.default.catalog_sales +(8) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] @@ -93,206 +88,226 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) -(12) ColumnarToRow [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] +Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] +Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +(12) CometProject Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#19] +(13) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#19, d_quarter_name#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] +ReadSchema: struct + +(14) CometFilter +Input [2]: [d_date_sk#19, d_quarter_name#20] +Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND isnotnull(d_date_sk#19)) -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None +(15) CometProject +Input [2]: [d_date_sk#19, d_quarter_name#20] +Arguments: [d_date_sk#19], [d_date_sk#19] -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +(16) CometBroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: [d_date_sk#19] + +(17) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight + +(18) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -(19) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#20] +(19) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#21, d_quarter_name#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(20) CometFilter +Input [2]: [d_date_sk#21, d_quarter_name#22] +Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) + +(21) CometProject +Input [2]: [d_date_sk#21, d_quarter_name#22] +Arguments: [d_date_sk#21], [d_date_sk#21] -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(22) CometBroadcastExchange +Input [1]: [d_date_sk#21] +Arguments: [d_date_sk#21] -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] +(23) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#21] +Arguments: [sr_returned_date_sk#12], [d_date_sk#21], Inner, BuildRight -(22) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#21] +(24) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None +(25) ReusedExchange [Reuses operator id: 22] +Output [1]: [d_date_sk#23] -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] +(26) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#23] +Arguments: [cs_sold_date_sk#17], [d_date_sk#23], Inner, BuildRight -(25) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_state#23] +(27) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#23] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] + +(28) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(26) CometFilter -Input [2]: [s_store_sk#22, s_state#23] -Condition : isnotnull(s_store_sk#22) - -(27) ColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#22, s_state#23] +(29) CometFilter +Input [2]: [s_store_sk#24, s_state#25] +Condition : isnotnull(s_store_sk#24) -(28) BroadcastExchange -Input [2]: [s_store_sk#22, s_state#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(30) CometBroadcastExchange +Input [2]: [s_store_sk#24, s_state#25] +Arguments: [s_store_sk#24, s_state#25] -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] +Right output [2]: [s_store_sk#24, s_state#25] +Arguments: [ss_store_sk#3], [s_store_sk#24], Inner, BuildRight -(30) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_state#23] +(32) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#24, s_state#25] +Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25] -(31) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] +(33) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Condition : isnotnull(i_item_sk#24) +(34) CometFilter +Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] +Condition : isnotnull(i_item_sk#26) -(33) ColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] +(35) CometBroadcastExchange +Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] +Arguments: [i_item_sk#26, i_item_id#27, i_item_desc#28] -(34) BroadcastExchange -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(36) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25] +Right output [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] +Arguments: [ss_item_sk#1], [i_item_sk#26], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#24] -Join type: Inner -Join condition: None +(37) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_sk#26, i_item_id#27, i_item_desc#28] +Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28] -(36) Project [codegen id : 8] -Output [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#25, i_item_desc#26] -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_sk#24, i_item_id#25, i_item_desc#26] +(38) ColumnarToRow [codegen id : 1] +Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28] -(37) HashAggregate [codegen id : 8] -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#25, i_item_desc#26] -Keys [3]: [i_item_id#25, i_item_desc#26, s_state#23] +(39) HashAggregate [codegen id : 1] +Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28] +Keys [3]: [i_item_id#27, i_item_desc#28, s_state#25] Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [18]: [count#27, sum#28, count#29, n#30, avg#31, m2#32, count#33, sum#34, count#35, n#36, avg#37, m2#38, count#39, sum#40, count#41, n#42, avg#43, m2#44] -Results [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] +Aggregate Attributes [18]: [count#29, sum#30, count#31, n#32, avg#33, m2#34, count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46] +Results [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#47, sum#48, count#49, n#50, avg#51, m2#52, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64] -(38) Exchange -Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] -Arguments: hashpartitioning(i_item_id#25, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(40) Exchange +Input [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#47, sum#48, count#49, n#50, avg#51, m2#52, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64] +Arguments: hashpartitioning(i_item_id#27, i_item_desc#28, s_state#25, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(39) HashAggregate [codegen id : 9] -Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] -Keys [3]: [i_item_id#25, i_item_desc#26, s_state#23] +(41) HashAggregate [codegen id : 2] +Input [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#47, sum#48, count#49, n#50, avg#51, m2#52, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64] +Keys [3]: [i_item_id#27, i_item_desc#28, s_state#25] Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [9]: [count(ss_quantity#5)#63, avg(ss_quantity#5)#64, stddev_samp(cast(ss_quantity#5 as double))#65, count(sr_return_quantity#11)#66, avg(sr_return_quantity#11)#67, stddev_samp(cast(sr_return_quantity#11 as double))#68, count(cs_quantity#16)#69, avg(cs_quantity#16)#70, stddev_samp(cast(cs_quantity#16 as double))#71] -Results [15]: [i_item_id#25, i_item_desc#26, s_state#23, count(ss_quantity#5)#63 AS store_sales_quantitycount#72, avg(ss_quantity#5)#64 AS store_sales_quantityave#73, stddev_samp(cast(ss_quantity#5 as double))#65 AS store_sales_quantitystdev#74, (stddev_samp(cast(ss_quantity#5 as double))#65 / avg(ss_quantity#5)#64) AS store_sales_quantitycov#75, count(sr_return_quantity#11)#66 AS as_store_returns_quantitycount#76, avg(sr_return_quantity#11)#67 AS as_store_returns_quantityave#77, stddev_samp(cast(sr_return_quantity#11 as double))#68 AS as_store_returns_quantitystdev#78, (stddev_samp(cast(sr_return_quantity#11 as double))#68 / avg(sr_return_quantity#11)#67) AS store_returns_quantitycov#79, count(cs_quantity#16)#69 AS catalog_sales_quantitycount#80, avg(cs_quantity#16)#70 AS catalog_sales_quantityave#81, (stddev_samp(cast(cs_quantity#16 as double))#71 / avg(cs_quantity#16)#70) AS catalog_sales_quantitystdev#82, (stddev_samp(cast(cs_quantity#16 as double))#71 / avg(cs_quantity#16)#70) AS catalog_sales_quantitycov#83] +Aggregate Attributes [9]: [count(ss_quantity#5)#65, avg(ss_quantity#5)#66, stddev_samp(cast(ss_quantity#5 as double))#67, count(sr_return_quantity#11)#68, avg(sr_return_quantity#11)#69, stddev_samp(cast(sr_return_quantity#11 as double))#70, count(cs_quantity#16)#71, avg(cs_quantity#16)#72, stddev_samp(cast(cs_quantity#16 as double))#73] +Results [15]: [i_item_id#27, i_item_desc#28, s_state#25, count(ss_quantity#5)#65 AS store_sales_quantitycount#74, avg(ss_quantity#5)#66 AS store_sales_quantityave#75, stddev_samp(cast(ss_quantity#5 as double))#67 AS store_sales_quantitystdev#76, (stddev_samp(cast(ss_quantity#5 as double))#67 / avg(ss_quantity#5)#66) AS store_sales_quantitycov#77, count(sr_return_quantity#11)#68 AS as_store_returns_quantitycount#78, avg(sr_return_quantity#11)#69 AS as_store_returns_quantityave#79, stddev_samp(cast(sr_return_quantity#11 as double))#70 AS as_store_returns_quantitystdev#80, (stddev_samp(cast(sr_return_quantity#11 as double))#70 / avg(sr_return_quantity#11)#69) AS store_returns_quantitycov#81, count(cs_quantity#16)#71 AS catalog_sales_quantitycount#82, avg(cs_quantity#16)#72 AS catalog_sales_quantityave#83, (stddev_samp(cast(cs_quantity#16 as double))#73 / avg(cs_quantity#16)#72) AS catalog_sales_quantitystdev#84, (stddev_samp(cast(cs_quantity#16 as double))#73 / avg(cs_quantity#16)#72) AS catalog_sales_quantitycov#85] -(40) TakeOrderedAndProject -Input [15]: [i_item_id#25, i_item_desc#26, s_state#23, store_sales_quantitycount#72, store_sales_quantityave#73, store_sales_quantitystdev#74, store_sales_quantitycov#75, as_store_returns_quantitycount#76, as_store_returns_quantityave#77, as_store_returns_quantitystdev#78, store_returns_quantitycov#79, catalog_sales_quantitycount#80, catalog_sales_quantityave#81, catalog_sales_quantitystdev#82, catalog_sales_quantitycov#83] -Arguments: 100, [i_item_id#25 ASC NULLS FIRST, i_item_desc#26 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#25, i_item_desc#26, s_state#23, store_sales_quantitycount#72, store_sales_quantityave#73, store_sales_quantitystdev#74, store_sales_quantitycov#75, as_store_returns_quantitycount#76, as_store_returns_quantityave#77, as_store_returns_quantitystdev#78, store_returns_quantitycov#79, catalog_sales_quantitycount#80, catalog_sales_quantityave#81, catalog_sales_quantitystdev#82, catalog_sales_quantitycov#83] +(42) TakeOrderedAndProject +Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#74, store_sales_quantityave#75, store_sales_quantitystdev#76, store_sales_quantitycov#77, as_store_returns_quantitycount#78, as_store_returns_quantityave#79, as_store_returns_quantitystdev#80, store_returns_quantitycov#81, catalog_sales_quantitycount#82, catalog_sales_quantityave#83, catalog_sales_quantitystdev#84, catalog_sales_quantitycov#85] +Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_state#25 ASC NULLS FIRST], [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#74, store_sales_quantityave#75, store_sales_quantitystdev#76, store_sales_quantitycov#77, as_store_returns_quantitycount#78, as_store_returns_quantityave#79, as_store_returns_quantitystdev#80, store_returns_quantitycov#81, catalog_sales_quantitycount#82, catalog_sales_quantityave#83, catalog_sales_quantitystdev#84, catalog_sales_quantitycov#85] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(41) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#84] +(43) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#19, d_quarter_name#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#84] -Condition : ((isnotnull(d_quarter_name#84) AND (d_quarter_name#84 = 2001Q1)) AND isnotnull(d_date_sk#19)) +(44) CometFilter +Input [2]: [d_date_sk#19, d_quarter_name#20] +Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND isnotnull(d_date_sk#19)) -(43) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#84] +(45) CometProject +Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(44) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(45) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_quarter_name#85] +(48) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#21, d_quarter_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [2]: [d_date_sk#20, d_quarter_name#85] -Condition : (d_quarter_name#85 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#20)) +(49) CometFilter +Input [2]: [d_date_sk#21, d_quarter_name#22] +Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) -(48) CometProject -Input [2]: [d_date_sk#20, d_quarter_name#85] -Arguments: [d_date_sk#20], [d_date_sk#20] +(50) CometProject +Input [2]: [d_date_sk#21, d_quarter_name#22] +Arguments: [d_date_sk#21], [d_date_sk#21] -(49) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] +(51) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#21] -(50) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(52) BroadcastExchange +Input [1]: [d_date_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt index 9f4d67decc..5c9726a22f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - WholeStageCodegen (9) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] InputAdapter Exchange [i_item_id,i_item_desc,s_state] #1 - WholeStageCodegen (8) + WholeStageCodegen (1) 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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,46 +31,33 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s CometProject [d_date_sk] CometFilter [d_quarter_name,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometBroadcastExchange #3 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometBroadcastExchange #5 + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometBroadcastExchange #7 + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange #8 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange #9 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt index 1e9c660c55..acbc8e654d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt @@ -1,47 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Expand (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Project (17) - : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- CometProject (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : : +- BroadcastExchange (15) - : : : : +- * ColumnarToRow (14) - : : : : +- CometProject (13) - : : : : +- CometFilter (12) - : : : : +- CometScan parquet spark_catalog.default.customer (11) - : : : +- BroadcastExchange (21) - : : : +- * ColumnarToRow (20) - : : : +- CometFilter (19) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) - : : +- BroadcastExchange (27) - : : +- * ColumnarToRow (26) - : : +- CometFilter (25) - : : +- CometScan parquet spark_catalog.default.customer_address (24) - : +- ReusedExchange (30) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.item (33) +TakeOrderedAndProject (41) ++- * HashAggregate (40) + +- Exchange (39) + +- * HashAggregate (38) + +- * ColumnarToRow (37) + +- CometExpand (36) + +- CometProject (35) + +- CometBroadcastHashJoin (34) + :- CometProject (30) + : +- CometBroadcastHashJoin (29) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometProject (19) + : : : +- CometBroadcastHashJoin (18) + : : : :- CometProject (14) + : : : : +- CometBroadcastHashJoin (13) + : : : : :- CometProject (8) + : : : : : +- CometBroadcastHashJoin (7) + : : : : : :- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : +- CometBroadcastExchange (6) + : : : : : +- CometProject (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : : : +- CometBroadcastExchange (12) + : : : : +- CometProject (11) + : : : : +- CometFilter (10) + : : : : +- CometScan parquet spark_catalog.default.customer (9) + : : : +- CometBroadcastExchange (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (15) + : : +- CometBroadcastExchange (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.customer_address (20) + : +- CometBroadcastExchange (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.date_dim (25) + +- CometBroadcastExchange (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.item (31) (1) Scan parquet spark_catalog.default.catalog_sales @@ -56,226 +54,221 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) ColumnarToRow [codegen id : 1] -Input [2]: [cd_demo_sk#11, cd_dep_count#14] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#11, cd_dep_count#14] -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Right output [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +(8) CometProject Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(11) Scan parquet spark_catalog.default.customer +(9) Scan parquet spark_catalog.default.customer Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) -(13) CometProject +(11) CometProject Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(15) BroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(13) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +(14) CometProject Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(18) Scan parquet spark_catalog.default.customer_demographics +(15) Scan parquet spark_catalog.default.customer_demographics Output [1]: [cd_demo_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(19) CometFilter +(16) CometFilter Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) ColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#20] - -(21) BroadcastExchange +(17) CometBroadcastExchange Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [cd_demo_sk#20] -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [cd_demo_sk#20] +Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +(19) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(24) Scan parquet spark_catalog.default.customer_address +(20) Scan parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [IN,MS,ND,NM,OK,VA]), IsNotNull(ca_address_sk)] ReadSchema: struct -(25) CometFilter +(21) CometFilter Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) -(26) ColumnarToRow [codegen id : 4] -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] - -(27) BroadcastExchange +(22) CometBroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -(28) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight -(29) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +(24) CometProject Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] -(30) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#25] +(25) Scan parquet spark_catalog.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,1998), IsNotNull(d_date_sk)] +ReadSchema: struct + +(26) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : ((isnotnull(d_year#26) AND (d_year#26 = 1998)) AND isnotnull(d_date_sk#25)) -(31) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(27) CometProject +Input [2]: [d_date_sk#25, d_year#26] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(28) CometBroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: [d_date_sk#25] -(32) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +(29) CometBroadcastHashJoin +Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#9], [d_date_sk#25], Inner, BuildRight + +(30) CometProject Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, d_date_sk#25] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] -(33) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#26, i_item_id#27] +(31) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#27, i_item_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) CometFilter -Input [2]: [i_item_sk#26, i_item_id#27] -Condition : isnotnull(i_item_sk#26) +(32) CometFilter +Input [2]: [i_item_sk#27, i_item_id#28] +Condition : isnotnull(i_item_sk#27) -(35) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#26, i_item_id#27] +(33) CometBroadcastExchange +Input [2]: [i_item_sk#27, i_item_id#28] +Arguments: [i_item_sk#27, i_item_id#28] -(36) BroadcastExchange -Input [2]: [i_item_sk#26, i_item_id#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(34) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Right output [2]: [i_item_sk#27, i_item_id#28] +Arguments: [cs_item_sk#3], [i_item_sk#27], Inner, BuildRight -(37) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None +(35) CometProject +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#27, i_item_id#28] +Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22] -(38) Project [codegen id : 7] -Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, ca_county#22] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#26, i_item_id#27] +(36) CometExpand +Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22] +Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] -(39) Expand [codegen id : 7] -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, ca_county#22] -Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] +(37) ColumnarToRow [codegen id : 1] +Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] -(40) HashAggregate [codegen id : 7] -Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] -Keys [5]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] +(38) HashAggregate [codegen id : 1] +Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] +Keys [5]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] -Aggregate Attributes [14]: [sum#33, count#34, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46] -Results [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] +Aggregate Attributes [14]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47] +Results [19]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] -(41) Exchange -Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] -Arguments: hashpartitioning(i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(39) Exchange +Input [19]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] +Arguments: hashpartitioning(i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(42) HashAggregate [codegen id : 8] -Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] -Keys [5]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] +(40) HashAggregate [codegen id : 2] +Input [19]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] +Keys [5]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] -Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#61, avg(cast(cs_list_price#5 as decimal(12,2)))#62, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63, avg(cast(cs_sales_price#6 as decimal(12,2)))#64, avg(cast(cs_net_profit#8 as decimal(12,2)))#65, avg(cast(c_birth_year#19 as decimal(12,2)))#66, avg(cast(cd_dep_count#14 as decimal(12,2)))#67] -Results [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, avg(cast(cs_quantity#4 as decimal(12,2)))#61 AS agg1#68, avg(cast(cs_list_price#5 as decimal(12,2)))#62 AS agg2#69, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63 AS agg3#70, avg(cast(cs_sales_price#6 as decimal(12,2)))#64 AS agg4#71, avg(cast(cs_net_profit#8 as decimal(12,2)))#65 AS agg5#72, avg(cast(c_birth_year#19 as decimal(12,2)))#66 AS agg6#73, avg(cast(cd_dep_count#14 as decimal(12,2)))#67 AS agg7#74] +Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#62, avg(cast(cs_list_price#5 as decimal(12,2)))#63, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#64, avg(cast(cs_sales_price#6 as decimal(12,2)))#65, avg(cast(cs_net_profit#8 as decimal(12,2)))#66, avg(cast(c_birth_year#19 as decimal(12,2)))#67, avg(cast(cd_dep_count#14 as decimal(12,2)))#68] +Results [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, avg(cast(cs_quantity#4 as decimal(12,2)))#62 AS agg1#69, avg(cast(cs_list_price#5 as decimal(12,2)))#63 AS agg2#70, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#64 AS agg3#71, avg(cast(cs_sales_price#6 as decimal(12,2)))#65 AS agg4#72, avg(cast(cs_net_profit#8 as decimal(12,2)))#66 AS agg5#73, avg(cast(c_birth_year#19 as decimal(12,2)))#67 AS agg6#74, avg(cast(cd_dep_count#14 as decimal(12,2)))#68 AS agg7#75] -(43) TakeOrderedAndProject -Input [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] -Arguments: 100, [ca_country#29 ASC NULLS FIRST, ca_state#30 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] +(41) TakeOrderedAndProject +Input [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#69, agg2#70, agg3#71, agg4#72, agg5#73, agg6#74, agg7#75] +Arguments: 100, [ca_country#30 ASC NULLS FIRST, ca_state#31 ASC NULLS FIRST, ca_county#32 ASC NULLS FIRST, i_item_id#29 ASC NULLS FIRST], [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#69, agg2#70, agg3#71, agg4#72, agg5#73, agg6#74, agg7#75] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) -(44) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#75] +(42) Scan parquet spark_catalog.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,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [2]: [d_date_sk#25, d_year#75] -Condition : ((isnotnull(d_year#75) AND (d_year#75 = 1998)) AND isnotnull(d_date_sk#25)) +(43) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : ((isnotnull(d_year#26) AND (d_year#26 = 1998)) AND isnotnull(d_date_sk#25)) -(46) CometProject -Input [2]: [d_date_sk#25, d_year#75] +(44) CometProject +Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(47) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(48) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt index 47911b9ba3..561ad3d4fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] + CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -30,42 +30,24 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange #5 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange #6 + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange #7 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #8 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt index 999fec8383..0353b1c2fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt @@ -1,43 +1,38 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (17) - : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.date_dim (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan parquet spark_catalog.default.store_sales (5) - : : : +- BroadcastExchange (15) - : : : +- * ColumnarToRow (14) - : : : +- CometProject (13) - : : : +- CometFilter (12) - : : : +- CometScan parquet spark_catalog.default.item (11) - : : +- BroadcastExchange (21) - : : +- * ColumnarToRow (20) - : : +- CometFilter (19) - : : +- CometScan parquet spark_catalog.default.customer (18) - : +- BroadcastExchange (27) - : +- * ColumnarToRow (26) - : +- CometFilter (25) - : +- CometScan parquet spark_catalog.default.customer_address (24) - +- BroadcastExchange (33) - +- * ColumnarToRow (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.store (30) +TakeOrderedAndProject (34) ++- * HashAggregate (33) + +- Exchange (32) + +- * HashAggregate (31) + +- * ColumnarToRow (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (14) + : : : +- CometBroadcastHashJoin (13) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (12) + : : : +- CometProject (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.item (9) + : : +- CometBroadcastExchange (17) + : : +- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.customer (15) + : +- CometBroadcastExchange (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.customer_address (20) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.store (25) (1) Scan parquet spark_catalog.default.date_dim @@ -55,10 +50,7 @@ Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) Input [3]: [d_date_sk#1, d_year#2, d_moy#3] Arguments: [d_date_sk#1], [d_date_sk#1] -(4) ColumnarToRow [codegen id : 6] -Input [1]: [d_date_sk#1] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -66,162 +58,145 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) -(7) ColumnarToRow [codegen id : 1] -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[4, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#8] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [1]: [d_date_sk#1] +Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] +Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight -(10) Project [codegen id : 6] -Output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] +(8) CometProject Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] +Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] -(11) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] 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 -(12) CometFilter +(10) CometFilter Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) -(13) CometProject +(11) CometProject Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] Arguments: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Arguments: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -(15) BroadcastExchange -Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(13) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] +Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#4] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 6] -Output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +(14) CometProject Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -(18) Scan parquet spark_catalog.default.customer +(15) Scan parquet spark_catalog.default.customer Output [2]: [c_customer_sk#15, c_current_addr_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(19) CometFilter +(16) CometFilter Input [2]: [c_customer_sk#15, c_current_addr_sk#16] Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) -(20) ColumnarToRow [codegen id : 3] -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] - -(21) BroadcastExchange +(17) CometBroadcastExchange Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [c_customer_sk#15, c_current_addr_sk#16] -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#5] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Right output [2]: [c_customer_sk#15, c_current_addr_sk#16] +Arguments: [ss_customer_sk#5], [c_customer_sk#15], Inner, BuildRight -(23) Project [codegen id : 6] -Output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16] +(19) CometProject Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_customer_sk#15, c_current_addr_sk#16] +Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16] -(24) Scan parquet spark_catalog.default.customer_address +(20) Scan parquet spark_catalog.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 -(25) CometFilter +(21) CometFilter Input [2]: [ca_address_sk#17, ca_zip#18] Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_zip#18)) -(26) ColumnarToRow [codegen id : 4] +(22) CometBroadcastExchange Input [2]: [ca_address_sk#17, ca_zip#18] +Arguments: [ca_address_sk#17, ca_zip#18] -(27) BroadcastExchange -Input [2]: [ca_address_sk#17, ca_zip#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_addr_sk#16] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16] +Right output [2]: [ca_address_sk#17, ca_zip#18] +Arguments: [c_current_addr_sk#16], [ca_address_sk#17], Inner, BuildRight -(29) Project [codegen id : 6] -Output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18] +(24) CometProject Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16, ca_address_sk#17, ca_zip#18] +Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18] -(30) Scan parquet spark_catalog.default.store +(25) Scan parquet spark_catalog.default.store Output [2]: [s_store_sk#19, s_zip#20] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_zip), IsNotNull(s_store_sk)] ReadSchema: struct -(31) CometFilter +(26) CometFilter Input [2]: [s_store_sk#19, s_zip#20] Condition : (isnotnull(s_zip#20) AND isnotnull(s_store_sk#19)) -(32) ColumnarToRow [codegen id : 5] -Input [2]: [s_store_sk#19, s_zip#20] - -(33) BroadcastExchange +(27) CometBroadcastExchange Input [2]: [s_store_sk#19, s_zip#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [s_store_sk#19, s_zip#20] -(34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#19] -Join type: Inner -Join condition: NOT (substr(ca_zip#18, 1, 5) = substr(s_zip#20, 1, 5)) +(28) CometBroadcastHashJoin +Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18] +Right output [2]: [s_store_sk#19, s_zip#20] +Arguments: [ss_store_sk#6], [s_store_sk#19], Inner, NOT (substr(ca_zip#18, 1, 5) = substr(s_zip#20, 1, 5)), BuildRight -(35) Project [codegen id : 6] -Output [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +(29) CometProject Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18, s_store_sk#19, s_zip#20] +Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] + +(30) ColumnarToRow [codegen id : 1] +Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -(36) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 1] Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] Keys [4]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] Aggregate Attributes [1]: [sum#21] Results [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] -(37) Exchange +(32) Exchange Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] -Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(38) HashAggregate [codegen id : 7] +(33) HashAggregate [codegen id : 2] Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] Keys [4]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#23] Results [5]: [i_brand_id#10 AS brand_id#24, i_brand#11 AS brand#25, i_manufact_id#12, i_manufact#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#23,17,2) AS ext_price#26] -(39) TakeOrderedAndProject +(34) TakeOrderedAndProject Input [5]: [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] Arguments: 100, [ext_price#26 DESC NULLS LAST, brand#25 ASC NULLS FIRST, brand_id#24 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt index c2f5d1a876..852b61d1c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt @@ -1,58 +1,38 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] - WholeStageCodegen (7) + WholeStageCodegen (2) HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - WholeStageCodegen (6) + WholeStageCodegen (1) 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] - BroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] - Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [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] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [s_zip,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] + CometBroadcastExchange #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometBroadcastExchange #4 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #5 + CometFilter [ca_address_sk,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange #6 + CometFilter [s_zip,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt index 1215adf3f0..48b744d962 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt @@ -1,40 +1,39 @@ == Physical Plan == -* Sort (36) -+- Exchange (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * ColumnarToRow (6) - : : : +- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.date_dim (7) - : +- BroadcastExchange (20) - : +- * ColumnarToRow (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.date_dim (16) - +- BroadcastExchange (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * HashAggregate (24) - : +- ReusedExchange (23) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +* Sort (35) ++- Exchange (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * HashAggregate (14) + : : +- Exchange (13) + : : +- * HashAggregate (12) + : : +- * ColumnarToRow (11) + : : +- CometProject (10) + : : +- CometBroadcastHashJoin (9) + : : :- CometUnion (5) + : : : :- CometProject (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- CometProject (4) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) + : : +- CometBroadcastExchange (8) + : : +- CometFilter (7) + : : +- CometScan parquet spark_catalog.default.date_dim (6) + : +- BroadcastExchange (19) + : +- * ColumnarToRow (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.date_dim (15) + +- BroadcastExchange (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * HashAggregate (23) + : +- ReusedExchange (22) + +- BroadcastExchange (28) + +- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) (1) Scan parquet spark_catalog.default.web_sales @@ -63,148 +62,144 @@ Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7 Child 0 Input [2]: [sold_date_sk#3, sales_price#4] Child 1 Input [2]: [sold_date_sk#7, sales_price#8] -(6) ColumnarToRow [codegen id : 2] -Input [2]: [sold_date_sk#3, sales_price#4] - -(7) Scan parquet spark_catalog.default.date_dim +(6) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] ReadSchema: struct -(8) CometFilter +(7) CometFilter Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) -(9) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] - -(10) BroadcastExchange +(8) CometBroadcastExchange Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#11] -(11) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [sold_date_sk#3] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometBroadcastHashJoin +Left output [2]: [sold_date_sk#3, sales_price#4] +Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] +Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight -(12) Project [codegen id : 2] -Output [3]: [sales_price#4, d_week_seq#10, d_day_name#11] +(10) CometProject Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#11] +Arguments: [sales_price#4, d_week_seq#10, d_day_name#11], [sales_price#4, d_week_seq#10, d_day_name#11] -(13) HashAggregate [codegen id : 2] +(11) ColumnarToRow [codegen id : 1] +Input [3]: [sales_price#4, d_week_seq#10, d_day_name#11] + +(12) HashAggregate [codegen id : 1] Input [3]: [sales_price#4, d_week_seq#10, d_day_name#11] Keys [1]: [d_week_seq#10] Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] Aggregate Attributes [7]: [sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18] Results [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] -(14) Exchange +(13) Exchange Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 8] +(14) HashAggregate [codegen id : 6] Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32] Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26,17,2) AS sun_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27,17,2) AS mon_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28,17,2) AS tue_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29,17,2) AS wed_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30,17,2) AS thu_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31,17,2) AS fri_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32,17,2) AS sat_sales#39] -(16) Scan parquet spark_catalog.default.date_dim +(15) Scan parquet spark_catalog.default.date_dim Output [2]: [d_week_seq#40, d_year#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] ReadSchema: struct -(17) CometFilter +(16) CometFilter Input [2]: [d_week_seq#40, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2001)) AND isnotnull(d_week_seq#40)) -(18) CometProject +(17) CometProject Input [2]: [d_week_seq#40, d_year#41] Arguments: [d_week_seq#40], [d_week_seq#40] -(19) ColumnarToRow [codegen id : 3] +(18) ColumnarToRow [codegen id : 2] Input [1]: [d_week_seq#40] -(20) BroadcastExchange +(19) BroadcastExchange Input [1]: [d_week_seq#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(21) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [d_week_seq#10] Right keys [1]: [d_week_seq#40] Join type: Inner Join condition: None -(22) Project [codegen id : 8] +(21) Project [codegen id : 6] Output [8]: [d_week_seq#10 AS d_week_seq1#42, sun_sales#33 AS sun_sales1#43, mon_sales#34 AS mon_sales1#44, tue_sales#35 AS tue_sales1#45, wed_sales#36 AS wed_sales1#46, thu_sales#37 AS thu_sales1#47, fri_sales#38 AS fri_sales1#48, sat_sales#39 AS sat_sales1#49] Input [9]: [d_week_seq#10, sun_sales#33, mon_sales#34, tue_sales#35, wed_sales#36, thu_sales#37, fri_sales#38, sat_sales#39, d_week_seq#40] -(23) ReusedExchange [Reuses operator id: 14] +(22) ReusedExchange [Reuses operator id: 13] Output [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57] -(24) HashAggregate [codegen id : 7] +(23) HashAggregate [codegen id : 5] Input [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57] Keys [1]: [d_week_seq#50] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))] Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))#32] Results [8]: [d_week_seq#50, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END))#26,17,2) AS sun_sales#60, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END))#27,17,2) AS mon_sales#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END))#28,17,2) AS tue_sales#62, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END))#29,17,2) AS wed_sales#63, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END))#30,17,2) AS thu_sales#64, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END))#31,17,2) AS fri_sales#65, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))#32,17,2) AS sat_sales#66] -(25) Scan parquet spark_catalog.default.date_dim +(24) Scan parquet spark_catalog.default.date_dim Output [2]: [d_week_seq#67, d_year#68] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct -(26) CometFilter +(25) CometFilter Input [2]: [d_week_seq#67, d_year#68] Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) -(27) CometProject +(26) CometProject Input [2]: [d_week_seq#67, d_year#68] Arguments: [d_week_seq#67], [d_week_seq#67] -(28) ColumnarToRow [codegen id : 6] +(27) ColumnarToRow [codegen id : 4] Input [1]: [d_week_seq#67] -(29) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_week_seq#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 7] +(29) BroadcastHashJoin [codegen id : 5] Left keys [1]: [d_week_seq#50] Right keys [1]: [d_week_seq#67] Join type: Inner Join condition: None -(31) Project [codegen id : 7] +(30) Project [codegen id : 5] Output [8]: [d_week_seq#50 AS d_week_seq2#69, sun_sales#60 AS sun_sales2#70, mon_sales#61 AS mon_sales2#71, tue_sales#62 AS tue_sales2#72, wed_sales#63 AS wed_sales2#73, thu_sales#64 AS thu_sales2#74, fri_sales#65 AS fri_sales2#75, sat_sales#66 AS sat_sales2#76] Input [9]: [d_week_seq#50, sun_sales#60, mon_sales#61, tue_sales#62, wed_sales#63, thu_sales#64, fri_sales#65, sat_sales#66, d_week_seq#67] -(32) BroadcastExchange +(31) BroadcastExchange Input [8]: [d_week_seq2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 6] Left keys [1]: [d_week_seq1#42] Right keys [1]: [(d_week_seq2#69 - 53)] Join type: Inner Join condition: None -(34) Project [codegen id : 8] +(33) Project [codegen id : 6] Output [8]: [d_week_seq1#42, round((sun_sales1#43 / sun_sales2#70), 2) AS round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1#44 / mon_sales2#71), 2) AS round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1#45 / tue_sales2#72), 2) AS round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1#46 / wed_sales2#73), 2) AS round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1#47 / thu_sales2#74), 2) AS round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1#48 / fri_sales2#75), 2) AS round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1#49 / sat_sales2#76), 2) AS round((sat_sales1 / sat_sales2), 2)#83] Input [16]: [d_week_seq1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] -(35) Exchange +(34) Exchange Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1 / sat_sales2), 2)#83] -Arguments: rangepartitioning(d_week_seq1#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: rangepartitioning(d_week_seq1#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(36) Sort [codegen id : 9] +(35) Sort [codegen id : 7] Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1 / sat_sales2), 2)#83] Arguments: [d_week_seq1#42 ASC NULLS FIRST], true, 0 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt index 8856ce80d2..334c6f0eab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (9) +WholeStageCodegen (7) Sort [d_week_seq1] InputAdapter Exchange [d_week_seq1] #1 - WholeStageCodegen (8) + WholeStageCodegen (6) Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] BroadcastHashJoin [d_week_seq1,d_week_seq2] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] @@ -10,27 +10,23 @@ WholeStageCodegen (9) HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [sold_date_sk,d_date_sk] CometUnion CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange #3 + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) + WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_week_seq] @@ -38,7 +34,7 @@ WholeStageCodegen (9) CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) + WholeStageCodegen (5) Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] @@ -46,7 +42,7 @@ WholeStageCodegen (9) ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) + WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt index 333ef218c6..4bb7f5ca5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * HashAggregate (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.catalog_sales @@ -33,118 +35,129 @@ ReadSchema: struct Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [cs_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] +Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(15) HashAggregate [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Aggregate Attributes [1]: [sum#13] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] -(14) Exchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +(17) HashAggregate [codegen id : 2] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] -Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#15] +Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#15,17,2) AS itemrevenue#16, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#15,17,2) AS _w0#17, i_item_id#6] -(16) Exchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(18) Exchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +(19) Sort [codegen id : 3] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(20) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] +Arguments: [sum(_w0#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9] -(19) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] -Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] +(21) Project [codegen id : 4] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, ((_w0#17 * 100) / _we0#18) AS revenueratio#19, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6, _we0#18] -(20) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +(22) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#19 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) -(21) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(23) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt index 52c42bdf2b..dd1edb95fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,12 +29,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt index 77f17c7f36..b2f81dbc1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt @@ -4,25 +4,25 @@ TakeOrderedAndProject (24) +- * HashAggregate (22) +- Exchange (21) +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.warehouse (4) - : +- BroadcastExchange (14) - : +- * ColumnarToRow (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.item (10) - +- ReusedExchange (17) + +- * ColumnarToRow (19) + +- CometProject (18) + +- CometBroadcastHashJoin (17) + :- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (7) + : : +- CometBroadcastHashJoin (6) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- CometBroadcastExchange (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.warehouse (3) + : +- CometBroadcastExchange (11) + : +- CometProject (10) + : +- CometFilter (9) + : +- CometScan parquet spark_catalog.default.item (8) + +- CometBroadcastExchange (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.date_dim (14) (1) Scan parquet spark_catalog.default.inventory @@ -37,83 +37,86 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Condition : isnotnull(w_warehouse_sk#6) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [w_warehouse_sk#6, w_warehouse_name#7] -(7) BroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#6], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#6] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] +(7) CometProject Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7], [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] -(10) Scan parquet spark_catalog.default.item +(8) Scan parquet spark_catalog.default.item Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) -(12) CometProject +(10) CometProject Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] Arguments: [i_item_sk#8, i_item_id#9], [i_item_sk#8, i_item_id#9] -(13) ColumnarToRow [codegen id : 2] +(11) CometBroadcastExchange Input [2]: [i_item_sk#8, i_item_id#9] +Arguments: [i_item_sk#8, i_item_id#9] -(14) BroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(12) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] +Right output [2]: [i_item_sk#8, i_item_id#9] +Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 4] -Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9] +(13) CometProject Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#9] +Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9], [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9] -(17) ReusedExchange [Reuses operator id: 28] +(14) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] +ReadSchema: struct + +(15) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-10)) AND (d_date#12 <= 2000-04-10)) AND isnotnull(d_date_sk#11)) -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(16) CometBroadcastExchange +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11, d_date#12] + +(17) CometBroadcastHashJoin +Left output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9] +Right output [2]: [d_date_sk#11, d_date#12] +Arguments: [inv_date_sk#4], [d_date_sk#11], Inner, BuildRight -(19) Project [codegen id : 4] -Output [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] +(18) CometProject Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9, d_date_sk#11, d_date#12] +Arguments: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12], [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] + +(19) ColumnarToRow [codegen id : 1] +Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] Keys [2]: [w_warehouse_name#7, i_item_id#9] Functions [2]: [partial_sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] @@ -122,16 +125,16 @@ Results [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] (21) Exchange Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] +(22) HashAggregate [codegen id : 2] Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] Keys [2]: [w_warehouse_name#7, i_item_id#9] Functions [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] Aggregate Attributes [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18] Results [4]: [w_warehouse_name#7, i_item_id#9, sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17 AS inv_before#19, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_after#20] -(23) Filter [codegen id : 5] +(23) Filter [codegen id : 2] Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] Condition : (CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667) END AND CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5) END) @@ -164,6 +167,6 @@ Input [2]: [d_date_sk#11, d_date#12] (28) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt index e20755e12f..c6058d18f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - WholeStageCodegen (5) + WholeStageCodegen (2) Filter [inv_before,inv_after] HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] InputAdapter Exchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] CometFilter [inv_warehouse_sk,inv_item_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -23,20 +23,13 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] InputAdapter CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #2 + CometBroadcastExchange #3 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #4 + CometProject [i_item_sk,i_item_id] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometBroadcastExchange #5 + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt index 9f5771fed2..342926b93d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt @@ -1,27 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.warehouse (13) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * ColumnarToRow (20) + +- CometExpand (19) + +- CometProject (18) + +- CometBroadcastHashJoin (17) + :- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.item (9) + +- CometBroadcastExchange (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.warehouse (14) (1) Scan parquet spark_catalog.default.inventory @@ -36,134 +37,141 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +(7) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] +Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -(7) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(9) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(9) ColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(10) CometFilter +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Condition : isnotnull(i_item_sk#8) -(10) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +Right output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight -(12) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(13) CometProject +Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(13) Scan parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#12] +(14) Scan parquet spark_catalog.default.warehouse +Output [1]: [w_warehouse_sk#13] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(14) CometFilter -Input [1]: [w_warehouse_sk#12] -Condition : isnotnull(w_warehouse_sk#12) +(15) CometFilter +Input [1]: [w_warehouse_sk#13] +Condition : isnotnull(w_warehouse_sk#13) -(15) ColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#12] +(16) CometBroadcastExchange +Input [1]: [w_warehouse_sk#13] +Arguments: [w_warehouse_sk#13] -(16) BroadcastExchange -Input [1]: [w_warehouse_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Right output [1]: [w_warehouse_sk#13] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#13], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#12] -Join type: Inner -Join condition: None +(18) CometProject +Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12, w_warehouse_sk#13] +Arguments: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11] -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11, w_warehouse_sk#12] +(19) CometExpand +Input [5]: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11] +Arguments: [[inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11, 0], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, null, 1], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, null, null, 3], [inv_quantity_on_hand#3, i_product_name#12, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] -(19) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10] -Arguments: [[inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10, 0], [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, null, 1], [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, null, null, 3], [inv_quantity_on_hand#3, i_product_name#11, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] +(20) ColumnarToRow [codegen id : 1] +Input [6]: [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] -(20) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#3, i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] -Keys [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] +(21) HashAggregate [codegen id : 1] +Input [6]: [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] +Keys [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#18, count#19] -Results [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] +Aggregate Attributes [2]: [sum#19, count#20] +Results [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#21, count#22] -(21) Exchange -Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] -Arguments: hashpartitioning(i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) Exchange +Input [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#21, count#22] +Arguments: hashpartitioning(i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] -Keys [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] +(23) HashAggregate [codegen id : 2] +Input [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#21, count#22] +Keys [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#22] -Results [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, avg(inv_quantity_on_hand#3)#22 AS qoh#23] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#23] +Results [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, avg(inv_quantity_on_hand#3)#23 AS qoh#24] -(23) TakeOrderedAndProject -Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] -Arguments: 100, [qoh#23 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] +(24) TakeOrderedAndProject +Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#24] +Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_category#17 ASC NULLS FIRST], [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#24] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * ColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.date_dim (24) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) -(24) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#24] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#24] -Condition : (((isnotnull(d_month_seq#24) AND (d_month_seq#24 >= 1200)) AND (d_month_seq#24 <= 1211)) AND isnotnull(d_date_sk#6)) +(26) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) -(26) CometProject -Input [2]: [d_date_sk#6, d_month_seq#24] +(27) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(27) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(28) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt index 92714bb02d..d89167ec8d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] + CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -24,19 +24,13 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt index 328a8d3536..10f78aaa18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt @@ -1,70 +1,71 @@ == Physical Plan == -* HashAggregate (66) -+- Exchange (65) - +- * HashAggregate (64) - +- Union (63) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * SortMergeJoin LeftSemi (41) - : : :- * Sort (24) - : : : +- Exchange (23) - : : : +- * Project (22) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (21) +* HashAggregate (67) ++- Exchange (66) + +- * HashAggregate (65) + +- Union (64) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (43) + : : +- * SortMergeJoin LeftSemi (42) + : : :- * Sort (26) + : : : +- Exchange (25) + : : : +- * Project (24) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (23) : : : :- * ColumnarToRow (2) : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (20) - : : : +- * Project (19) - : : : +- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- Exchange (16) - : : : +- * HashAggregate (15) - : : : +- * Project (14) - : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : :- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (12) - : : : +- * ColumnarToRow (11) - : : : +- CometFilter (10) - : : : +- CometScan parquet spark_catalog.default.item (9) - : : +- * Sort (40) - : : +- * Project (39) - : : +- * Filter (38) - : : +- * HashAggregate (37) - : : +- Exchange (36) - : : +- * HashAggregate (35) - : : +- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * ColumnarToRow (28) - : : : +- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometScan parquet spark_catalog.default.store_sales (25) - : : +- BroadcastExchange (32) - : : +- * ColumnarToRow (31) - : : +- CometFilter (30) - : : +- CometScan parquet spark_catalog.default.customer (29) - : +- ReusedExchange (43) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * SortMergeJoin LeftSemi (58) - : :- * Sort (52) - : : +- Exchange (51) - : : +- * Project (50) - : : +- * BroadcastHashJoin LeftSemi BuildRight (49) - : : :- * ColumnarToRow (47) - : : : +- CometScan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (48) - : +- * Sort (57) - : +- * Project (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- ReusedExchange (53) - +- ReusedExchange (60) + : : : +- BroadcastExchange (22) + : : : +- * Project (21) + : : : +- * Filter (20) + : : : +- * HashAggregate (19) + : : : +- Exchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometHashAggregate (16) + : : : +- CometProject (15) + : : : +- CometBroadcastHashJoin (14) + : : : :- CometProject (10) + : : : : +- CometBroadcastHashJoin (9) + : : : : :- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (8) + : : : : +- CometProject (7) + : : : : +- CometFilter (6) + : : : : +- CometScan parquet spark_catalog.default.date_dim (5) + : : : +- CometBroadcastExchange (13) + : : : +- CometFilter (12) + : : : +- CometScan parquet spark_catalog.default.item (11) + : : +- * Sort (41) + : : +- * Project (40) + : : +- * Filter (39) + : : +- * HashAggregate (38) + : : +- Exchange (37) + : : +- * HashAggregate (36) + : : +- * ColumnarToRow (35) + : : +- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan parquet spark_catalog.default.store_sales (27) + : : +- CometBroadcastExchange (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.customer (30) + : +- ReusedExchange (44) + +- * Project (63) + +- * BroadcastHashJoin Inner BuildRight (62) + :- * Project (60) + : +- * SortMergeJoin LeftSemi (59) + : :- * Sort (53) + : : +- Exchange (52) + : : +- * Project (51) + : : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : : :- * ColumnarToRow (48) + : : : +- CometScan parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (49) + : +- * Sort (58) + : +- * Project (57) + : +- * Filter (56) + : +- * HashAggregate (55) + : +- ReusedExchange (54) + +- ReusedExchange (61) (1) Scan parquet spark_catalog.default.catalog_sales @@ -74,7 +75,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 3] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (3) Scan parquet spark_catalog.default.store_sales @@ -89,287 +90,292 @@ ReadSchema: struct Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(5) ColumnarToRow [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] +(5) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_date#11, d_year#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 76] -Output [2]: [d_date_sk#10, d_date#11] +(6) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(7) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(7) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] + +(8) CometBroadcastExchange +Input [2]: [d_date_sk#10, d_date#11] +Arguments: [d_date_sk#10, d_date#11] + +(9) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] +Right output [2]: [d_date_sk#10, d_date#11] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight -(8) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#11] +(10) CometProject Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] +Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] -(9) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#12, i_item_desc#13] +(11) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#13, i_item_desc#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) - -(11) ColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] +(12) CometFilter +Input [2]: [i_item_sk#13, i_item_desc#14] +Condition : isnotnull(i_item_sk#13) -(12) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(13) CometBroadcastExchange +Input [2]: [i_item_sk#13, i_item_desc#14] +Arguments: [i_item_sk#13, i_item_desc#14] -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#7, d_date#11] +Right output [2]: [i_item_sk#13, i_item_desc#14] +Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight -(14) Project [codegen id : 3] -Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] +(15) CometProject +Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] +Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] -(15) HashAggregate [codegen id : 3] -Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +(16) CometHashAggregate +Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] +Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(16) Exchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(17) ColumnarToRow [codegen id : 1] +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -(17) HashAggregate [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +(18) Exchange +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] +Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(19) HashAggregate [codegen id : 2] +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] +Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] +Results [2]: [i_item_sk#13 AS item_sk#18, count(1)#17 AS cnt#19] -(18) Filter [codegen id : 4] +(20) Filter [codegen id : 2] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(19) Project [codegen id : 4] +(21) Project [codegen id : 2] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] -(20) BroadcastExchange +(22) BroadcastExchange Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(21) BroadcastHashJoin [codegen id : 5] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(22) Project [codegen id : 5] +(24) Project [codegen id : 3] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(23) Exchange +(25) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) Sort [codegen id : 6] +(26) Sort [codegen id : 4] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(25) Scan parquet spark_catalog.default.store_sales +(27) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(26) CometFilter +(28) CometFilter Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Condition : isnotnull(ss_customer_sk#20) -(27) CometProject +(29) CometProject Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -(28) ColumnarToRow [codegen id : 8] -Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(29) Scan parquet spark_catalog.default.customer +(30) Scan parquet spark_catalog.default.customer Output [1]: [c_customer_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(30) CometFilter +(31) CometFilter Input [1]: [c_customer_sk#24] Condition : isnotnull(c_customer_sk#24) -(31) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#24] - -(32) BroadcastExchange +(32) CometBroadcastExchange Input [1]: [c_customer_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#24] -(33) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#20] -Right keys [1]: [c_customer_sk#24] -Join type: Inner -Join condition: None +(33) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] +Right output [1]: [c_customer_sk#24] +Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight -(34) Project [codegen id : 8] -Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +(34) CometProject Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] + +(35) ColumnarToRow [codegen id : 5] +Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -(35) HashAggregate [codegen id : 8] +(36) HashAggregate [codegen id : 5] Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Keys [1]: [c_customer_sk#24] Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [2]: [sum#25, isEmpty#26] Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(36) Exchange +(37) Exchange Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(37) HashAggregate [codegen id : 9] +(38) HashAggregate [codegen id : 6] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(38) Filter [codegen id : 9] +(39) Filter [codegen id : 6] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) -(39) Project [codegen id : 9] +(40) Project [codegen id : 6] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(40) Sort [codegen id : 9] +(41) Sort [codegen id : 6] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(41) SortMergeJoin [codegen id : 11] +(42) SortMergeJoin [codegen id : 8] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(42) Project [codegen id : 11] +(43) Project [codegen id : 8] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(43) ReusedExchange [Reuses operator id: 71] +(44) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#33] -(44) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(45) Project [codegen id : 11] +(46) Project [codegen id : 8] Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#34] Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#33] -(46) Scan parquet spark_catalog.default.web_sales +(47) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#39), dynamicpruningexpression(ws_sold_date_sk#39 IN dynamicpruning#40)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] +(48) ColumnarToRow [codegen id : 11] Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(48) ReusedExchange [Reuses operator id: 20] +(49) ReusedExchange [Reuses operator id: 22] Output [1]: [item_sk#41] -(49) BroadcastHashJoin [codegen id : 16] +(50) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ws_item_sk#35] Right keys [1]: [item_sk#41] Join type: LeftSemi Join condition: None -(50) Project [codegen id : 16] +(51) Project [codegen id : 11] Output [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(51) Exchange +(52) Exchange Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -Arguments: hashpartitioning(ws_bill_customer_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(ws_bill_customer_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(52) Sort [codegen id : 17] +(53) Sort [codegen id : 12] Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Arguments: [ws_bill_customer_sk#36 ASC NULLS FIRST], false, 0 -(53) ReusedExchange [Reuses operator id: 36] +(54) ReusedExchange [Reuses operator id: 37] Output [3]: [c_customer_sk#42, sum#43, isEmpty#44] -(54) HashAggregate [codegen id : 20] +(55) HashAggregate [codegen id : 14] Input [3]: [c_customer_sk#42, sum#43, isEmpty#44] Keys [1]: [c_customer_sk#42] Functions [1]: [sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))] Aggregate Attributes [1]: [sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))#29] Results [2]: [c_customer_sk#42, sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))#29 AS ssales#47] -(55) Filter [codegen id : 20] +(56) Filter [codegen id : 14] Input [2]: [c_customer_sk#42, ssales#47] Condition : (isnotnull(ssales#47) AND (cast(ssales#47 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(56) Project [codegen id : 20] +(57) Project [codegen id : 14] Output [1]: [c_customer_sk#42] Input [2]: [c_customer_sk#42, ssales#47] -(57) Sort [codegen id : 20] +(58) Sort [codegen id : 14] Input [1]: [c_customer_sk#42] Arguments: [c_customer_sk#42 ASC NULLS FIRST], false, 0 -(58) SortMergeJoin [codegen id : 22] +(59) SortMergeJoin [codegen id : 16] Left keys [1]: [ws_bill_customer_sk#36] Right keys [1]: [c_customer_sk#42] Join type: LeftSemi Join condition: None -(59) Project [codegen id : 22] +(60) Project [codegen id : 16] Output [3]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -(60) ReusedExchange [Reuses operator id: 71] +(61) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#48] -(61) BroadcastHashJoin [codegen id : 22] +(62) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_sold_date_sk#39] Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(62) Project [codegen id : 22] +(63) Project [codegen id : 16] Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#49] Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#48] -(63) Union +(64) Union -(64) HashAggregate [codegen id : 23] +(65) HashAggregate [codegen id : 17] Input [1]: [sales#34] Keys: [] Functions [1]: [partial_sum(sales#34)] Aggregate Attributes [2]: [sum#50, isEmpty#51] Results [2]: [sum#52, isEmpty#53] -(65) Exchange +(66) Exchange Input [2]: [sum#52, isEmpty#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(66) HashAggregate [codegen id : 24] +(67) HashAggregate [codegen id : 18] Input [2]: [sum#52, isEmpty#53] Keys: [] Functions [1]: [sum(sales#34)] @@ -379,192 +385,209 @@ Results [1]: [sum(sales#34)#54 AS sum(sales)#55] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (71) -+- * ColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan parquet spark_catalog.default.date_dim (67) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(67) Scan parquet spark_catalog.default.date_dim +(68) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#33, d_year#56, d_moy#57] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(68) CometFilter +(69) CometFilter Input [3]: [d_date_sk#33, d_year#56, d_moy#57] Condition : ((((isnotnull(d_year#56) AND isnotnull(d_moy#57)) AND (d_year#56 = 2000)) AND (d_moy#57 = 2)) AND isnotnull(d_date_sk#33)) -(69) CometProject +(70) CometProject Input [3]: [d_date_sk#33, d_year#56, d_moy#57] Arguments: [d_date_sk#33], [d_date_sk#33] -(70) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#33] -(71) BroadcastExchange +(72) BroadcastExchange Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (77) ++- * ColumnarToRow (76) + +- CometProject (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(72) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#58] +(73) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_date#11, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#58] -Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +(74) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(74) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#58] +(75) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(75) ColumnarToRow [codegen id : 1] +(76) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(76) BroadcastExchange +(77) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (91) -+- Exchange (90) - +- * HashAggregate (89) - +- * HashAggregate (88) - +- Exchange (87) - +- * HashAggregate (86) - +- * Project (85) - +- * BroadcastHashJoin Inner BuildRight (84) - :- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * ColumnarToRow (79) - : : +- CometFilter (78) - : : +- CometScan parquet spark_catalog.default.store_sales (77) - : +- ReusedExchange (80) - +- ReusedExchange (83) - - -(77) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +* HashAggregate (95) ++- Exchange (94) + +- * HashAggregate (93) + +- * HashAggregate (92) + +- Exchange (91) + +- * HashAggregate (90) + +- * ColumnarToRow (89) + +- CometProject (88) + +- CometBroadcastHashJoin (87) + :- CometProject (82) + : +- CometBroadcastHashJoin (81) + : :- CometFilter (79) + : : +- CometScan parquet spark_catalog.default.store_sales (78) + : +- ReusedExchange (80) + +- CometBroadcastExchange (86) + +- CometProject (85) + +- CometFilter (84) + +- CometScan parquet spark_catalog.default.date_dim (83) + + +(78) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#62), dynamicpruningexpression(ss_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#62)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(78) CometFilter -Input [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] -Condition : isnotnull(ss_customer_sk#59) - -(79) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] +(79) CometFilter +Input [4]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61] +Condition : isnotnull(ss_customer_sk#58) (80) ReusedExchange [Reuses operator id: 32] -Output [1]: [c_customer_sk#64] +Output [1]: [c_customer_sk#63] -(81) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#59] -Right keys [1]: [c_customer_sk#64] -Join type: Inner -Join condition: None +(81) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61] +Right output [1]: [c_customer_sk#63] +Arguments: [ss_customer_sk#58], [c_customer_sk#63], Inner, BuildRight + +(82) CometProject +Input [5]: [ss_customer_sk#58, ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61, c_customer_sk#63] +Arguments: [ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61, c_customer_sk#63], [ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61, c_customer_sk#63] -(82) Project [codegen id : 3] -Output [4]: [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64] -Input [5]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64] +(83) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#64, d_year#65] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct -(83) ReusedExchange [Reuses operator id: 96] -Output [1]: [d_date_sk#65] +(84) CometFilter +Input [2]: [d_date_sk#64, d_year#65] +Condition : (d_year#65 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#64)) -(84) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#62] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None +(85) CometProject +Input [2]: [d_date_sk#64, d_year#65] +Arguments: [d_date_sk#64], [d_date_sk#64] + +(86) CometBroadcastExchange +Input [1]: [d_date_sk#64] +Arguments: [d_date_sk#64] + +(87) CometBroadcastHashJoin +Left output [4]: [ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61, c_customer_sk#63] +Right output [1]: [d_date_sk#64] +Arguments: [ss_sold_date_sk#61], [d_date_sk#64], Inner, BuildRight + +(88) CometProject +Input [5]: [ss_quantity#59, ss_sales_price#60, ss_sold_date_sk#61, c_customer_sk#63, d_date_sk#64] +Arguments: [ss_quantity#59, ss_sales_price#60, c_customer_sk#63], [ss_quantity#59, ss_sales_price#60, c_customer_sk#63] -(85) Project [codegen id : 3] -Output [3]: [ss_quantity#60, ss_sales_price#61, c_customer_sk#64] -Input [5]: [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64, d_date_sk#65] +(89) ColumnarToRow [codegen id : 1] +Input [3]: [ss_quantity#59, ss_sales_price#60, c_customer_sk#63] -(86) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#60, ss_sales_price#61, c_customer_sk#64] -Keys [1]: [c_customer_sk#64] -Functions [1]: [partial_sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))] +(90) HashAggregate [codegen id : 1] +Input [3]: [ss_quantity#59, ss_sales_price#60, c_customer_sk#63] +Keys [1]: [c_customer_sk#63] +Functions [1]: [partial_sum((cast(ss_quantity#59 as decimal(10,0)) * ss_sales_price#60))] Aggregate Attributes [2]: [sum#66, isEmpty#67] -Results [3]: [c_customer_sk#64, sum#68, isEmpty#69] +Results [3]: [c_customer_sk#63, sum#68, isEmpty#69] -(87) Exchange -Input [3]: [c_customer_sk#64, sum#68, isEmpty#69] -Arguments: hashpartitioning(c_customer_sk#64, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(91) Exchange +Input [3]: [c_customer_sk#63, sum#68, isEmpty#69] +Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(88) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#64, sum#68, isEmpty#69] -Keys [1]: [c_customer_sk#64] -Functions [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))#70] -Results [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))#70 AS csales#71] +(92) HashAggregate [codegen id : 2] +Input [3]: [c_customer_sk#63, sum#68, isEmpty#69] +Keys [1]: [c_customer_sk#63] +Functions [1]: [sum((cast(ss_quantity#59 as decimal(10,0)) * ss_sales_price#60))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#59 as decimal(10,0)) * ss_sales_price#60))#70] +Results [1]: [sum((cast(ss_quantity#59 as decimal(10,0)) * ss_sales_price#60))#70 AS csales#71] -(89) HashAggregate [codegen id : 4] +(93) HashAggregate [codegen id : 2] Input [1]: [csales#71] Keys: [] Functions [1]: [partial_max(csales#71)] Aggregate Attributes [1]: [max#72] Results [1]: [max#73] -(90) Exchange +(94) Exchange Input [1]: [max#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(91) HashAggregate [codegen id : 5] +(95) HashAggregate [codegen id : 3] Input [1]: [max#73] Keys: [] Functions [1]: [max(csales#71)] Aggregate Attributes [1]: [max(csales#71)#74] Results [1]: [max(csales#71)#74 AS tpcds_cmax#75] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#62 IN dynamicpruning#63 -BroadcastExchange (96) -+- * ColumnarToRow (95) - +- CometProject (94) - +- CometFilter (93) - +- CometScan parquet spark_catalog.default.date_dim (92) +Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#62 +BroadcastExchange (100) ++- * ColumnarToRow (99) + +- CometProject (98) + +- CometFilter (97) + +- CometScan parquet spark_catalog.default.date_dim (96) -(92) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_year#76] +(96) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#64, d_year#65] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(93) CometFilter -Input [2]: [d_date_sk#65, d_year#76] -Condition : (d_year#76 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#65)) +(97) CometFilter +Input [2]: [d_date_sk#64, d_year#65] +Condition : (d_year#65 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#64)) -(94) CometProject -Input [2]: [d_date_sk#65, d_year#76] -Arguments: [d_date_sk#65], [d_date_sk#65] +(98) CometProject +Input [2]: [d_date_sk#64, d_year#65] +Arguments: [d_date_sk#64], [d_date_sk#64] -(95) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] +(99) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#64] -(96) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(100) BroadcastExchange +Input [1]: [d_date_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:5 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:6 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt index 0ec56d0e72..1ed007bc79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt @@ -1,22 +1,22 @@ -WholeStageCodegen (24) +WholeStageCodegen (18) HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] InputAdapter Exchange #1 - WholeStageCodegen (23) + WholeStageCodegen (17) HashAggregate [sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (11) + WholeStageCodegen (8) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_quantity,cs_list_price,cs_sold_date_sk] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (4) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) + WholeStageCodegen (3) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow @@ -32,20 +32,20 @@ WholeStageCodegen (24) CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [item_sk] Filter [cnt] HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] InputAdapter Exchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 @@ -56,84 +56,79 @@ WholeStageCodegen (24) CometProject [d_date_sk,d_date] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange #7 + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange #8 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - WholeStageCodegen (9) + WholeStageCodegen (6) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] Subquery #3 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [max] [max(csales),tpcds_cmax,max] InputAdapter - Exchange #10 - WholeStageCodegen (4) + Exchange #11 + WholeStageCodegen (2) HashAggregate [csales] [max,max] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] InputAdapter - Exchange [c_customer_sk] #11 - WholeStageCodegen (3) + Exchange [c_customer_sk] #12 + WholeStageCodegen (1) HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 + BroadcastExchange #13 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [c_customer_sk] #9 - InputAdapter - ReusedExchange [d_date_sk] #12 + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange #14 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter - Exchange [c_customer_sk] #8 - WholeStageCodegen (8) + Exchange [c_customer_sk] #9 + WholeStageCodegen (5) HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange #10 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) + WholeStageCodegen (16) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_quantity,ws_list_price,ws_sold_date_sk] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (12) Sort [ws_bill_customer_sk] InputAdapter - Exchange [ws_bill_customer_sk] #13 - WholeStageCodegen (16) + Exchange [ws_bill_customer_sk] #15 + WholeStageCodegen (11) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow @@ -143,13 +138,13 @@ WholeStageCodegen (24) InputAdapter ReusedExchange [item_sk] #4 InputAdapter - WholeStageCodegen (20) + WholeStageCodegen (14) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] ReusedSubquery [tpcds_cmax] #3 HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [c_customer_sk,sum,isEmpty] #9 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt index 840f9734aa..2e6ff8aaad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt @@ -1,91 +1,92 @@ == Physical Plan == -TakeOrderedAndProject (87) -+- Union (86) - :- * HashAggregate (62) - : +- Exchange (61) - : +- * HashAggregate (60) - : +- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (56) - : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * SortMergeJoin LeftSemi (42) - : : : :- * Sort (25) - : : : : +- Exchange (24) - : : : : +- * Project (23) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (22) +TakeOrderedAndProject (88) ++- Union (87) + :- * HashAggregate (63) + : +- Exchange (62) + : +- * HashAggregate (61) + : +- * Project (60) + : +- * BroadcastHashJoin Inner BuildRight (59) + : :- * Project (57) + : : +- * BroadcastHashJoin Inner BuildRight (56) + : : :- * SortMergeJoin LeftSemi (43) + : : : :- * Sort (27) + : : : : +- Exchange (26) + : : : : +- * Project (25) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (24) : : : : :- * ColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (21) - : : : : +- * Project (20) - : : : : +- * Filter (19) - : : : : +- * HashAggregate (18) - : : : : +- Exchange (17) - : : : : +- * HashAggregate (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (7) - : : : : +- BroadcastExchange (13) - : : : : +- * ColumnarToRow (12) - : : : : +- CometFilter (11) - : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : +- * Sort (41) - : : : +- * Project (40) - : : : +- * Filter (39) - : : : +- * HashAggregate (38) - : : : +- Exchange (37) - : : : +- * HashAggregate (36) - : : : +- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (29) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan parquet spark_catalog.default.store_sales (26) - : : : +- BroadcastExchange (33) - : : : +- * ColumnarToRow (32) - : : : +- CometFilter (31) - : : : +- CometScan parquet spark_catalog.default.customer (30) - : : +- BroadcastExchange (54) - : : +- * SortMergeJoin LeftSemi (53) - : : :- * Sort (47) - : : : +- Exchange (46) - : : : +- * ColumnarToRow (45) - : : : +- CometFilter (44) - : : : +- CometScan parquet spark_catalog.default.customer (43) - : : +- * Sort (52) - : : +- * Project (51) - : : +- * Filter (50) - : : +- * HashAggregate (49) - : : +- ReusedExchange (48) - : +- ReusedExchange (57) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * BroadcastHashJoin Inner BuildRight (81) - :- * Project (79) - : +- * BroadcastHashJoin Inner BuildRight (78) - : :- * SortMergeJoin LeftSemi (76) - : : :- * Sort (70) - : : : +- Exchange (69) - : : : +- * Project (68) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (67) - : : : :- * ColumnarToRow (65) - : : : : +- CometFilter (64) - : : : : +- CometScan parquet spark_catalog.default.web_sales (63) - : : : +- ReusedExchange (66) - : : +- * Sort (75) - : : +- * Project (74) - : : +- * Filter (73) - : : +- * HashAggregate (72) - : : +- ReusedExchange (71) - : +- ReusedExchange (77) - +- ReusedExchange (80) + : : : : +- BroadcastExchange (23) + : : : : +- * Project (22) + : : : : +- * Filter (21) + : : : : +- * HashAggregate (20) + : : : : +- Exchange (19) + : : : : +- * ColumnarToRow (18) + : : : : +- CometHashAggregate (17) + : : : : +- CometProject (16) + : : : : +- CometBroadcastHashJoin (15) + : : : : :- CometProject (11) + : : : : : +- CometBroadcastHashJoin (10) + : : : : : :- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : : +- CometBroadcastExchange (9) + : : : : : +- CometProject (8) + : : : : : +- CometFilter (7) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (6) + : : : : +- CometBroadcastExchange (14) + : : : : +- CometFilter (13) + : : : : +- CometScan parquet spark_catalog.default.item (12) + : : : +- * Sort (42) + : : : +- * Project (41) + : : : +- * Filter (40) + : : : +- * HashAggregate (39) + : : : +- Exchange (38) + : : : +- * HashAggregate (37) + : : : +- * ColumnarToRow (36) + : : : +- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometProject (30) + : : : : +- CometFilter (29) + : : : : +- CometScan parquet spark_catalog.default.store_sales (28) + : : : +- CometBroadcastExchange (33) + : : : +- CometFilter (32) + : : : +- CometScan parquet spark_catalog.default.customer (31) + : : +- BroadcastExchange (55) + : : +- * SortMergeJoin LeftSemi (54) + : : :- * Sort (48) + : : : +- Exchange (47) + : : : +- * ColumnarToRow (46) + : : : +- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.customer (44) + : : +- * Sort (53) + : : +- * Project (52) + : : +- * Filter (51) + : : +- * HashAggregate (50) + : : +- ReusedExchange (49) + : +- ReusedExchange (58) + +- * HashAggregate (86) + +- Exchange (85) + +- * HashAggregate (84) + +- * Project (83) + +- * BroadcastHashJoin Inner BuildRight (82) + :- * Project (80) + : +- * BroadcastHashJoin Inner BuildRight (79) + : :- * SortMergeJoin LeftSemi (77) + : : :- * Sort (71) + : : : +- Exchange (70) + : : : +- * Project (69) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (68) + : : : :- * ColumnarToRow (66) + : : : : +- CometFilter (65) + : : : : +- CometScan parquet spark_catalog.default.web_sales (64) + : : : +- ReusedExchange (67) + : : +- * Sort (76) + : : +- * Project (75) + : : +- * Filter (74) + : : +- * HashAggregate (73) + : : +- ReusedExchange (72) + : +- ReusedExchange (78) + +- ReusedExchange (81) (1) Scan parquet spark_catalog.default.catalog_sales @@ -100,7 +101,7 @@ ReadSchema: struct Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(6) ColumnarToRow [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] +(6) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_date#11, d_year#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct -(7) ReusedExchange [Reuses operator id: 97] -Output [2]: [d_date_sk#10, d_date#11] +(7) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(8) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] + +(9) CometBroadcastExchange +Input [2]: [d_date_sk#10, d_date#11] +Arguments: [d_date_sk#10, d_date#11] + +(10) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] +Right output [2]: [d_date_sk#10, d_date#11] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight -(9) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#11] +(11) CometProject Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] +Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] -(10) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#12, i_item_desc#13] +(12) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#13, i_item_desc#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(11) CometFilter -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) - -(12) ColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] +(13) CometFilter +Input [2]: [i_item_sk#13, i_item_desc#14] +Condition : isnotnull(i_item_sk#13) -(13) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(14) CometBroadcastExchange +Input [2]: [i_item_sk#13, i_item_desc#14] +Arguments: [i_item_sk#13, i_item_desc#14] -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join type: Inner -Join condition: None +(15) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#7, d_date#11] +Right output [2]: [i_item_sk#13, i_item_desc#14] +Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight -(15) Project [codegen id : 3] -Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] +(16) CometProject +Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] +Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] -(16) HashAggregate [codegen id : 3] -Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +(17) CometHashAggregate +Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] +Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(17) Exchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(18) ColumnarToRow [codegen id : 1] +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -(18) HashAggregate [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +(19) Exchange +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] +Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(20) HashAggregate [codegen id : 2] +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] +Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] +Results [2]: [i_item_sk#13 AS item_sk#18, count(1)#17 AS cnt#19] -(19) Filter [codegen id : 4] +(21) Filter [codegen id : 2] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(20) Project [codegen id : 4] +(22) Project [codegen id : 2] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] -(21) BroadcastExchange +(23) BroadcastExchange Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(22) BroadcastHashJoin [codegen id : 5] +(24) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(23) Project [codegen id : 5] +(25) Project [codegen id : 3] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(24) Exchange +(26) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(25) Sort [codegen id : 6] +(27) Sort [codegen id : 4] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(26) Scan parquet spark_catalog.default.store_sales +(28) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(27) CometFilter +(29) CometFilter Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Condition : isnotnull(ss_customer_sk#20) -(28) CometProject +(30) CometProject Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -(29) ColumnarToRow [codegen id : 8] -Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(30) Scan parquet spark_catalog.default.customer +(31) Scan parquet spark_catalog.default.customer Output [1]: [c_customer_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [1]: [c_customer_sk#24] Condition : isnotnull(c_customer_sk#24) -(32) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#24] - -(33) BroadcastExchange +(33) CometBroadcastExchange Input [1]: [c_customer_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: [c_customer_sk#24] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#20] -Right keys [1]: [c_customer_sk#24] -Join type: Inner -Join condition: None +(34) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] +Right output [1]: [c_customer_sk#24] +Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight -(35) Project [codegen id : 8] -Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +(35) CometProject Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] + +(36) ColumnarToRow [codegen id : 5] +Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -(36) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 5] Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Keys [1]: [c_customer_sk#24] Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [2]: [sum#25, isEmpty#26] Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(37) Exchange +(38) Exchange Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(38) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 6] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(39) Filter [codegen id : 9] +(40) Filter [codegen id : 6] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) -(40) Project [codegen id : 9] +(41) Project [codegen id : 6] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(41) Sort [codegen id : 9] +(42) Sort [codegen id : 6] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(42) SortMergeJoin [codegen id : 17] +(43) SortMergeJoin [codegen id : 13] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(43) Scan parquet spark_catalog.default.customer +(44) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(44) CometFilter +(45) CometFilter Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] Condition : isnotnull(c_customer_sk#33) -(45) ColumnarToRow [codegen id : 10] +(46) ColumnarToRow [codegen id : 7] Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -(46) Exchange +(47) Exchange Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: hashpartitioning(c_customer_sk#33, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(c_customer_sk#33, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(47) Sort [codegen id : 11] +(48) Sort [codegen id : 8] Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] Arguments: [c_customer_sk#33 ASC NULLS FIRST], false, 0 -(48) ReusedExchange [Reuses operator id: 37] +(49) ReusedExchange [Reuses operator id: 38] Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(49) HashAggregate [codegen id : 14] +(50) HashAggregate [codegen id : 10] Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] -(50) Filter [codegen id : 14] +(51) Filter [codegen id : 10] Input [2]: [c_customer_sk#24, ssales#30] Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(51) Project [codegen id : 14] +(52) Project [codegen id : 10] Output [1]: [c_customer_sk#24] Input [2]: [c_customer_sk#24, ssales#30] -(52) Sort [codegen id : 14] +(53) Sort [codegen id : 10] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(53) SortMergeJoin [codegen id : 15] +(54) SortMergeJoin [codegen id : 11] Left keys [1]: [c_customer_sk#33] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(54) BroadcastExchange +(55) BroadcastExchange Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(55) BroadcastHashJoin [codegen id : 17] +(56) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#33] Join type: Inner Join condition: None -(56) Project [codegen id : 17] +(57) Project [codegen id : 13] Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35] Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#33, c_first_name#34, c_last_name#35] -(57) ReusedExchange [Reuses operator id: 92] +(58) ReusedExchange [Reuses operator id: 93] Output [1]: [d_date_sk#36] -(58) BroadcastHashJoin [codegen id : 17] +(59) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(60) Project [codegen id : 13] Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35, d_date_sk#36] -(60) HashAggregate [codegen id : 17] +(61) HashAggregate [codegen id : 13] Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] Aggregate Attributes [2]: [sum#37, isEmpty#38] Results [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -(61) Exchange +(62) Exchange Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(62) HashAggregate [codegen id : 18] +(63) HashAggregate [codegen id : 14] Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] Keys [2]: [c_last_name#35, c_first_name#34] Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] Aggregate Attributes [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41] Results [3]: [c_last_name#35, c_first_name#34, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41 AS sales#42] -(63) Scan parquet spark_catalog.default.web_sales +(64) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] @@ -392,303 +398,320 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) CometFilter +(65) CometFilter Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Condition : isnotnull(ws_bill_customer_sk#44) -(65) ColumnarToRow [codegen id : 23] +(66) ColumnarToRow [codegen id : 17] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(66) ReusedExchange [Reuses operator id: 21] +(67) ReusedExchange [Reuses operator id: 23] Output [1]: [item_sk#49] -(67) BroadcastHashJoin [codegen id : 23] +(68) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#49] Join type: LeftSemi Join condition: None -(68) Project [codegen id : 23] +(69) Project [codegen id : 17] Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -(69) Exchange +(70) Exchange Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(70) Sort [codegen id : 24] +(71) Sort [codegen id : 18] Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 -(71) ReusedExchange [Reuses operator id: 37] +(72) ReusedExchange [Reuses operator id: 38] Output [3]: [c_customer_sk#50, sum#51, isEmpty#52] -(72) HashAggregate [codegen id : 27] +(73) HashAggregate [codegen id : 20] Input [3]: [c_customer_sk#50, sum#51, isEmpty#52] Keys [1]: [c_customer_sk#50] Functions [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] Aggregate Attributes [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#29] Results [2]: [c_customer_sk#50, sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#29 AS ssales#55] -(73) Filter [codegen id : 27] +(74) Filter [codegen id : 20] Input [2]: [c_customer_sk#50, ssales#55] Condition : (isnotnull(ssales#55) AND (cast(ssales#55 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) -(74) Project [codegen id : 27] +(75) Project [codegen id : 20] Output [1]: [c_customer_sk#50] Input [2]: [c_customer_sk#50, ssales#55] -(75) Sort [codegen id : 27] +(76) Sort [codegen id : 20] Input [1]: [c_customer_sk#50] Arguments: [c_customer_sk#50 ASC NULLS FIRST], false, 0 -(76) SortMergeJoin [codegen id : 35] +(77) SortMergeJoin [codegen id : 27] Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#50] Join type: LeftSemi Join condition: None -(77) ReusedExchange [Reuses operator id: 54] +(78) ReusedExchange [Reuses operator id: 55] Output [3]: [c_customer_sk#56, c_first_name#57, c_last_name#58] -(78) BroadcastHashJoin [codegen id : 35] +(79) BroadcastHashJoin [codegen id : 27] Left keys [1]: [ws_bill_customer_sk#44] Right keys [1]: [c_customer_sk#56] Join type: Inner Join condition: None -(79) Project [codegen id : 35] +(80) Project [codegen id : 27] Output [5]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#57, c_last_name#58] Input [7]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_customer_sk#56, c_first_name#57, c_last_name#58] -(80) ReusedExchange [Reuses operator id: 92] +(81) ReusedExchange [Reuses operator id: 93] Output [1]: [d_date_sk#59] -(81) BroadcastHashJoin [codegen id : 35] +(82) BroadcastHashJoin [codegen id : 27] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#59] Join type: Inner Join condition: None -(82) Project [codegen id : 35] +(83) Project [codegen id : 27] Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#57, c_last_name#58] Input [6]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#57, c_last_name#58, d_date_sk#59] -(83) HashAggregate [codegen id : 35] +(84) HashAggregate [codegen id : 27] Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#57, c_last_name#58] Keys [2]: [c_last_name#58, c_first_name#57] Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] Aggregate Attributes [2]: [sum#60, isEmpty#61] Results [4]: [c_last_name#58, c_first_name#57, sum#62, isEmpty#63] -(84) Exchange +(85) Exchange Input [4]: [c_last_name#58, c_first_name#57, sum#62, isEmpty#63] -Arguments: hashpartitioning(c_last_name#58, c_first_name#57, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(c_last_name#58, c_first_name#57, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(85) HashAggregate [codegen id : 36] +(86) HashAggregate [codegen id : 28] Input [4]: [c_last_name#58, c_first_name#57, sum#62, isEmpty#63] Keys [2]: [c_last_name#58, c_first_name#57] Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#64] Results [3]: [c_last_name#58, c_first_name#57, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#64 AS sales#65] -(86) Union +(87) Union -(87) TakeOrderedAndProject +(88) TakeOrderedAndProject Input [3]: [c_last_name#35, c_first_name#34, sales#42] Arguments: 100, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, sales#42 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, sales#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (92) -+- * ColumnarToRow (91) - +- CometProject (90) - +- CometFilter (89) - +- CometScan parquet spark_catalog.default.date_dim (88) +BroadcastExchange (93) ++- * ColumnarToRow (92) + +- CometProject (91) + +- CometFilter (90) + +- CometScan parquet spark_catalog.default.date_dim (89) -(88) Scan parquet spark_catalog.default.date_dim +(89) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#36, d_year#66, d_moy#67] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(89) CometFilter +(90) CometFilter Input [3]: [d_date_sk#36, d_year#66, d_moy#67] Condition : ((((isnotnull(d_year#66) AND isnotnull(d_moy#67)) AND (d_year#66 = 2000)) AND (d_moy#67 = 2)) AND isnotnull(d_date_sk#36)) -(90) CometProject +(91) CometProject Input [3]: [d_date_sk#36, d_year#66, d_moy#67] Arguments: [d_date_sk#36], [d_date_sk#36] -(91) ColumnarToRow [codegen id : 1] +(92) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#36] -(92) BroadcastExchange +(93) BroadcastExchange Input [1]: [d_date_sk#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (97) -+- * ColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometScan parquet spark_catalog.default.date_dim (93) +BroadcastExchange (98) ++- * ColumnarToRow (97) + +- CometProject (96) + +- CometFilter (95) + +- CometScan parquet spark_catalog.default.date_dim (94) -(93) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#68] +(94) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_date#11, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(94) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#68] -Condition : (d_year#68 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +(95) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(95) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#68] +(96) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(96) ColumnarToRow [codegen id : 1] +(97) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(97) BroadcastExchange +(98) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (112) -+- Exchange (111) - +- * HashAggregate (110) - +- * HashAggregate (109) - +- Exchange (108) - +- * HashAggregate (107) - +- * Project (106) - +- * BroadcastHashJoin Inner BuildRight (105) - :- * Project (103) - : +- * BroadcastHashJoin Inner BuildRight (102) - : :- * ColumnarToRow (100) - : : +- CometFilter (99) - : : +- CometScan parquet spark_catalog.default.store_sales (98) - : +- ReusedExchange (101) - +- ReusedExchange (104) - - -(98) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#69, ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] + +Subquery:3 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +* HashAggregate (116) ++- Exchange (115) + +- * HashAggregate (114) + +- * HashAggregate (113) + +- Exchange (112) + +- * HashAggregate (111) + +- * ColumnarToRow (110) + +- CometProject (109) + +- CometBroadcastHashJoin (108) + :- CometProject (103) + : +- CometBroadcastHashJoin (102) + : :- CometFilter (100) + : : +- CometScan parquet spark_catalog.default.store_sales (99) + : +- ReusedExchange (101) + +- CometBroadcastExchange (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) + + +(99) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#68, ss_quantity#69, ss_sales_price#70, ss_sold_date_sk#71] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#72), dynamicpruningexpression(ss_sold_date_sk#72 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ss_sold_date_sk#71), dynamicpruningexpression(ss_sold_date_sk#71 IN dynamicpruning#72)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(99) CometFilter -Input [4]: [ss_customer_sk#69, ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72] -Condition : isnotnull(ss_customer_sk#69) - -(100) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#69, ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72] +(100) CometFilter +Input [4]: [ss_customer_sk#68, ss_quantity#69, ss_sales_price#70, ss_sold_date_sk#71] +Condition : isnotnull(ss_customer_sk#68) (101) ReusedExchange [Reuses operator id: 33] -Output [1]: [c_customer_sk#74] +Output [1]: [c_customer_sk#73] -(102) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#69] -Right keys [1]: [c_customer_sk#74] -Join type: Inner -Join condition: None +(102) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#68, ss_quantity#69, ss_sales_price#70, ss_sold_date_sk#71] +Right output [1]: [c_customer_sk#73] +Arguments: [ss_customer_sk#68], [c_customer_sk#73], Inner, BuildRight + +(103) CometProject +Input [5]: [ss_customer_sk#68, ss_quantity#69, ss_sales_price#70, ss_sold_date_sk#71, c_customer_sk#73] +Arguments: [ss_quantity#69, ss_sales_price#70, ss_sold_date_sk#71, c_customer_sk#73], [ss_quantity#69, ss_sales_price#70, ss_sold_date_sk#71, c_customer_sk#73] -(103) Project [codegen id : 3] -Output [4]: [ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72, c_customer_sk#74] -Input [5]: [ss_customer_sk#69, ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72, c_customer_sk#74] +(104) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#74, d_year#75] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct -(104) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#75] +(105) CometFilter +Input [2]: [d_date_sk#74, d_year#75] +Condition : (d_year#75 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#74)) -(105) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#72] -Right keys [1]: [d_date_sk#75] -Join type: Inner -Join condition: None +(106) CometProject +Input [2]: [d_date_sk#74, d_year#75] +Arguments: [d_date_sk#74], [d_date_sk#74] + +(107) CometBroadcastExchange +Input [1]: [d_date_sk#74] +Arguments: [d_date_sk#74] + +(108) CometBroadcastHashJoin +Left output [4]: [ss_quantity#69, ss_sales_price#70, ss_sold_date_sk#71, c_customer_sk#73] +Right output [1]: [d_date_sk#74] +Arguments: [ss_sold_date_sk#71], [d_date_sk#74], Inner, BuildRight + +(109) CometProject +Input [5]: [ss_quantity#69, ss_sales_price#70, ss_sold_date_sk#71, c_customer_sk#73, d_date_sk#74] +Arguments: [ss_quantity#69, ss_sales_price#70, c_customer_sk#73], [ss_quantity#69, ss_sales_price#70, c_customer_sk#73] -(106) Project [codegen id : 3] -Output [3]: [ss_quantity#70, ss_sales_price#71, c_customer_sk#74] -Input [5]: [ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72, c_customer_sk#74, d_date_sk#75] +(110) ColumnarToRow [codegen id : 1] +Input [3]: [ss_quantity#69, ss_sales_price#70, c_customer_sk#73] -(107) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#70, ss_sales_price#71, c_customer_sk#74] -Keys [1]: [c_customer_sk#74] -Functions [1]: [partial_sum((cast(ss_quantity#70 as decimal(10,0)) * ss_sales_price#71))] +(111) HashAggregate [codegen id : 1] +Input [3]: [ss_quantity#69, ss_sales_price#70, c_customer_sk#73] +Keys [1]: [c_customer_sk#73] +Functions [1]: [partial_sum((cast(ss_quantity#69 as decimal(10,0)) * ss_sales_price#70))] Aggregate Attributes [2]: [sum#76, isEmpty#77] -Results [3]: [c_customer_sk#74, sum#78, isEmpty#79] +Results [3]: [c_customer_sk#73, sum#78, isEmpty#79] -(108) Exchange -Input [3]: [c_customer_sk#74, sum#78, isEmpty#79] -Arguments: hashpartitioning(c_customer_sk#74, 5), ENSURE_REQUIREMENTS, [plan_id=14] +(112) Exchange +Input [3]: [c_customer_sk#73, sum#78, isEmpty#79] +Arguments: hashpartitioning(c_customer_sk#73, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(109) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#74, sum#78, isEmpty#79] -Keys [1]: [c_customer_sk#74] -Functions [1]: [sum((cast(ss_quantity#70 as decimal(10,0)) * ss_sales_price#71))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#70 as decimal(10,0)) * ss_sales_price#71))#80] -Results [1]: [sum((cast(ss_quantity#70 as decimal(10,0)) * ss_sales_price#71))#80 AS csales#81] +(113) HashAggregate [codegen id : 2] +Input [3]: [c_customer_sk#73, sum#78, isEmpty#79] +Keys [1]: [c_customer_sk#73] +Functions [1]: [sum((cast(ss_quantity#69 as decimal(10,0)) * ss_sales_price#70))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#69 as decimal(10,0)) * ss_sales_price#70))#80] +Results [1]: [sum((cast(ss_quantity#69 as decimal(10,0)) * ss_sales_price#70))#80 AS csales#81] -(110) HashAggregate [codegen id : 4] +(114) HashAggregate [codegen id : 2] Input [1]: [csales#81] Keys: [] Functions [1]: [partial_max(csales#81)] Aggregate Attributes [1]: [max#82] Results [1]: [max#83] -(111) Exchange +(115) Exchange Input [1]: [max#83] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(112) HashAggregate [codegen id : 5] +(116) HashAggregate [codegen id : 3] Input [1]: [max#83] Keys: [] Functions [1]: [max(csales#81)] Aggregate Attributes [1]: [max(csales#81)#84] Results [1]: [max(csales#81)#84 AS tpcds_cmax#85] -Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#72 IN dynamicpruning#73 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#71 IN dynamicpruning#72 +BroadcastExchange (121) ++- * ColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan parquet spark_catalog.default.date_dim (117) -(113) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#75, d_year#86] +(117) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#74, d_year#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter -Input [2]: [d_date_sk#75, d_year#86] -Condition : (d_year#86 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#75)) +(118) CometFilter +Input [2]: [d_date_sk#74, d_year#75] +Condition : (d_year#75 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#74)) -(115) CometProject -Input [2]: [d_date_sk#75, d_year#86] -Arguments: [d_date_sk#75], [d_date_sk#75] +(119) CometProject +Input [2]: [d_date_sk#74, d_year#75] +Arguments: [d_date_sk#74], [d_date_sk#74] -(116) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#75] +(120) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#74] -(117) BroadcastExchange -Input [1]: [d_date_sk#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] +(121) BroadcastExchange +Input [1]: [d_date_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:5 Hosting operator id = 50 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:5 Hosting operator id = 51 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] -Subquery:6 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:7 Hosting operator id = 74 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt index 49ddeaef82..09a2bf7cec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (18) + WholeStageCodegen (14) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (17) + WholeStageCodegen (13) HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_list_price,c_first_name,c_last_name] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -12,11 +12,11 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (4) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) + WholeStageCodegen (3) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow @@ -33,20 +33,20 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [item_sk] Filter [cnt] HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] InputAdapter Exchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 @@ -57,101 +57,96 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] CometProject [d_date_sk,d_date] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange #7 + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange #8 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - WholeStageCodegen (9) + WholeStageCodegen (6) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] Subquery #3 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [max] [max(csales),tpcds_cmax,max] InputAdapter - Exchange #10 - WholeStageCodegen (4) + Exchange #11 + WholeStageCodegen (2) HashAggregate [csales] [max,max] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] InputAdapter - Exchange [c_customer_sk] #11 - WholeStageCodegen (3) + Exchange [c_customer_sk] #12 + WholeStageCodegen (1) HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 + BroadcastExchange #13 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [c_customer_sk] #9 - InputAdapter - ReusedExchange [d_date_sk] #12 + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange #14 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter - Exchange [c_customer_sk] #8 - WholeStageCodegen (8) + Exchange [c_customer_sk] #9 + WholeStageCodegen (5) HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange #10 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (15) + BroadcastExchange #15 + WholeStageCodegen (11) SortMergeJoin [c_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (8) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #14 - WholeStageCodegen (10) + Exchange [c_customer_sk] #16 + WholeStageCodegen (7) ColumnarToRow InputAdapter CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - WholeStageCodegen (14) + WholeStageCodegen (10) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] ReusedSubquery [tpcds_cmax] #3 HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [c_customer_sk,sum,isEmpty] #9 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (36) + WholeStageCodegen (28) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty] InputAdapter - Exchange [c_last_name,c_first_name] #15 - WholeStageCodegen (35) + Exchange [c_last_name,c_first_name] #17 + WholeStageCodegen (27) HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_list_price,c_first_name,c_last_name] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -159,11 +154,11 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (18) Sort [ws_bill_customer_sk] InputAdapter - Exchange [ws_bill_customer_sk] #16 - WholeStageCodegen (23) + Exchange [ws_bill_customer_sk] #18 + WholeStageCodegen (17) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow @@ -174,15 +169,15 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter ReusedExchange [item_sk] #4 InputAdapter - WholeStageCodegen (27) + WholeStageCodegen (20) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] ReusedSubquery [tpcds_cmax] #3 HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [c_customer_sk,sum,isEmpty] #9 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt index 2635546e44..0ea8e3ef4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt @@ -1,44 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * ColumnarToRow (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometProject (18) + : : : : +- CometBroadcastHashJoin (17) + : : : : :- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (8) + : : : : +- CometBroadcastExchange (16) + : : : : +- CometProject (15) + : : : : +- CometFilter (14) + : : : : +- CometScan parquet spark_catalog.default.date_dim (13) + : : : +- CometBroadcastExchange (22) + : : : +- CometProject (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.date_dim (19) + : : +- ReusedExchange (25) + : +- CometBroadcastExchange (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.store (28) + +- CometBroadcastExchange (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.item (33) (1) Scan parquet spark_catalog.default.store_sales @@ -53,10 +55,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) -(6) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] +Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] +Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] +Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] +(7) CometProject Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] -(10) Scan parquet spark_catalog.default.catalog_sales +(8) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] @@ -93,206 +88,226 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) -(12) ColumnarToRow [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] +Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] +Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] +Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] +(12) CometProject Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#19] +(13) Scan parquet spark_catalog.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_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(14) CometFilter +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None +(15) CometProject +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Arguments: [d_date_sk#19], [d_date_sk#19] -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] +(16) CometBroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: [d_date_sk#19] + +(17) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight + +(18) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -(19) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#20] +(19) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#22, d_year#23, d_moy#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(20) CometFilter +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) + +(21) CometProject +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Arguments: [d_date_sk#22], [d_date_sk#22] -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(22) CometBroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: [d_date_sk#22] -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#20] +(23) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#22] +Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight -(22) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#21] +(24) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#22] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None +(25) ReusedExchange [Reuses operator id: 22] +Output [1]: [d_date_sk#25] -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#21] +(26) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight -(25) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +(27) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#25] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] + +(28) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(26) CometFilter -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Condition : isnotnull(s_store_sk#22) - -(27) ColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +(29) CometFilter +Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] +Condition : isnotnull(s_store_sk#26) -(28) BroadcastExchange -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(30) CometBroadcastExchange +Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] +Arguments: [s_store_sk#26, s_store_id#27, s_store_name#28] -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] +Right output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] +Arguments: [ss_store_sk#3], [s_store_sk#26], Inner, BuildRight -(30) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#22, s_store_id#23, s_store_name#24] +(32) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#26, s_store_id#27, s_store_name#28] +Arguments: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28], [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28] -(31) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +(33) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Condition : isnotnull(i_item_sk#25) +(34) CometFilter +Input [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] +Condition : isnotnull(i_item_sk#29) -(33) ColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +(35) CometBroadcastExchange +Input [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] +Arguments: [i_item_sk#29, i_item_id#30, i_item_desc#31] -(34) BroadcastExchange -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(36) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28] +Right output [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] +Arguments: [ss_item_sk#1], [i_item_sk#29], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#25] -Join type: Inner -Join condition: None +(37) CometProject +Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_sk#29, i_item_id#30, i_item_desc#31] +Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31] -(36) Project [codegen id : 8] -Output [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24, i_item_sk#25, i_item_id#26, i_item_desc#27] +(38) ColumnarToRow [codegen id : 1] +Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31] -(37) HashAggregate [codegen id : 8] -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] -Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +(39) HashAggregate [codegen id : 1] +Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31] +Keys [4]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28] Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum#28, sum#29, sum#30] -Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] +Aggregate Attributes [3]: [sum#32, sum#33, sum#34] +Results [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#35, sum#36, sum#37] -(38) Exchange -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(40) Exchange +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#35, sum#36, sum#37] +Arguments: hashpartitioning(i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(39) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +(41) HashAggregate [codegen id : 2] +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#35, sum#36, sum#37] +Keys [4]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#34, sum(UnscaledValue(sr_net_loss#11))#35, sum(UnscaledValue(cs_net_profit#16))#36] -Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#34,17,2) AS store_sales_profit#37, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#35,17,2) AS store_returns_loss#38, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#36,17,2) AS catalog_sales_profit#39] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#38, sum(UnscaledValue(sr_net_loss#11))#39, sum(UnscaledValue(cs_net_profit#16))#40] +Results [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#38,17,2) AS store_sales_profit#41, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#39,17,2) AS store_returns_loss#42, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#40,17,2) AS catalog_sales_profit#43] -(40) TakeOrderedAndProject -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] -Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] +(42) TakeOrderedAndProject +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#41, store_returns_loss#42, catalog_sales_profit#43] +Arguments: 100, [i_item_id#30 ASC NULLS FIRST, i_item_desc#31 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#41, store_returns_loss#42, catalog_sales_profit#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(41) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#40, d_moy#41] +(43) Scan parquet spark_catalog.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_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] -Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 4)) AND (d_year#40 = 2001)) AND isnotnull(d_date_sk#19)) +(44) CometFilter +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) -(43) CometProject -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] +(45) CometProject +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(44) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(45) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#42, d_moy#43] +(48) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#22, d_year#23, d_moy#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] -Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 4)) AND (d_moy#43 <= 10)) AND (d_year#42 = 2001)) AND isnotnull(d_date_sk#20)) +(49) CometFilter +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(48) CometProject -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] -Arguments: [d_date_sk#20], [d_date_sk#20] +(50) CometProject +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Arguments: [d_date_sk#22], [d_date_sk#22] -(49) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] +(51) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#22] -(50) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(52) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt index eda7f6b64d..41efffcb4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (9) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] InputAdapter Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) + WholeStageCodegen (1) 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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,46 +31,33 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales CometProject [d_date_sk] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometBroadcastExchange #3 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #7 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange #8 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange #9 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt index c90dcd0243..d0bc41bcf2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt @@ -3,29 +3,29 @@ TakeOrderedAndProject (30) +- * HashAggregate (29) +- Exchange (28) +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) + +- * ColumnarToRow (26) + +- CometProject (25) + +- CometBroadcastHashJoin (24) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.date_dim (9) + : +- CometBroadcastExchange (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.item (15) + +- CometBroadcastExchange (23) +- CometProject (22) +- CometFilter (21) +- CometScan parquet spark_catalog.default.promotion (20) @@ -43,135 +43,138 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) ColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#10] -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_bill_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] +Right output [1]: [cd_demo_sk#10] +Arguments: [cs_bill_cdemo_sk#1], [cd_demo_sk#10], Inner, BuildRight -(10) Project [codegen id : 5] -Output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] +(8) CometProject Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] +Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] + +(9) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +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 -(11) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#14] +(10) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(11) CometProject +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14], [d_date_sk#14] -(13) Project [codegen id : 5] -Output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] +Right output [1]: [d_date_sk#14] +Arguments: [cs_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] +Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] -(14) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] +(15) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#16, i_item_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : isnotnull(i_item_sk#15) - -(16) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#15, i_item_id#16] +(16) CometFilter +Input [2]: [i_item_sk#16, i_item_id#17] +Condition : isnotnull(i_item_sk#16) -(17) BroadcastExchange -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [2]: [i_item_sk#16, i_item_id#17] +Arguments: [i_item_sk#16, i_item_id#17] -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] +Right output [2]: [i_item_sk#16, i_item_id#17] +Arguments: [cs_item_sk#2], [i_item_sk#16], Inner, BuildRight -(19) Project [codegen id : 5] -Output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16] -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#15, i_item_id#16] +(19) CometProject +Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#16, i_item_id#17] +Arguments: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17], [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] (20) Scan parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] +Output [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [Or(EqualTo(p_channel_email,N),EqualTo(p_channel_event,N)), IsNotNull(p_promo_sk)] ReadSchema: struct (21) CometFilter -Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] -Condition : (((p_channel_email#18 = N) OR (p_channel_event#19 = N)) AND isnotnull(p_promo_sk#17)) +Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] +Condition : (((p_channel_email#19 = N) OR (p_channel_event#20 = N)) AND isnotnull(p_promo_sk#18)) (22) CometProject -Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] -Arguments: [p_promo_sk#17], [p_promo_sk#17] +Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] +Arguments: [p_promo_sk#18], [p_promo_sk#18] -(23) ColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#17] +(23) CometBroadcastExchange +Input [1]: [p_promo_sk#18] +Arguments: [p_promo_sk#18] -(24) BroadcastExchange -Input [1]: [p_promo_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(24) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] +Right output [1]: [p_promo_sk#18] +Arguments: [cs_promo_sk#3], [p_promo_sk#18], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_promo_sk#3] -Right keys [1]: [p_promo_sk#17] -Join type: Inner -Join condition: None +(25) CometProject +Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17, p_promo_sk#18] +Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -(26) Project [codegen id : 5] -Output [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16] -Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16, p_promo_sk#17] +(26) ColumnarToRow [codegen id : 1] +Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -(27) HashAggregate [codegen id : 5] -Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16] -Keys [1]: [i_item_id#16] +(27) HashAggregate [codegen id : 1] +Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] +Keys [1]: [i_item_id#17] Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [8]: [sum#20, count#21, sum#22, count#23, sum#24, count#25, sum#26, count#27] -Results [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Aggregate Attributes [8]: [sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] +Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] (28) Exchange -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] +Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [1]: [i_item_id#16] +(29) HashAggregate [codegen id : 2] +Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] +Keys [1]: [i_item_id#17] Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [4]: [avg(cs_quantity#4)#36, avg(UnscaledValue(cs_list_price#5))#37, avg(UnscaledValue(cs_coupon_amt#7))#38, avg(UnscaledValue(cs_sales_price#6))#39] -Results [5]: [i_item_id#16, avg(cs_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(cs_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(cs_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(cs_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] +Aggregate Attributes [4]: [avg(cs_quantity#4)#37, avg(UnscaledValue(cs_list_price#5))#38, avg(UnscaledValue(cs_coupon_amt#7))#39, avg(UnscaledValue(cs_sales_price#6))#40] +Results [5]: [i_item_id#17, avg(cs_quantity#4)#37 AS agg1#41, cast((avg(UnscaledValue(cs_list_price#5))#38 / 100.0) as decimal(11,6)) AS agg2#42, cast((avg(UnscaledValue(cs_coupon_amt#7))#39 / 100.0) as decimal(11,6)) AS agg3#43, cast((avg(UnscaledValue(cs_sales_price#6))#40 / 100.0) as decimal(11,6)) AS agg4#44] (30) TakeOrderedAndProject -Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] -Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] +Input [5]: [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] +Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] ===== Subqueries ===== @@ -184,18 +187,18 @@ BroadcastExchange (35) (31) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] 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 (32) CometFilter -Input [2]: [d_date_sk#14, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) (33) CometProject -Input [2]: [d_date_sk#14, d_year#44] +Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] (34) ColumnarToRow [codegen id : 1] @@ -203,6 +206,6 @@ Input [1]: [d_date_sk#14] (35) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt index 7d38936244..67f335c66f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] + CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -25,28 +25,18 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometBroadcastExchange #3 + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #5 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #6 + CometProject [p_promo_sk] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt index e41077ed30..16af0e3a43 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt @@ -3,30 +3,30 @@ TakeOrderedAndProject (30) +- * HashAggregate (29) +- Exchange (28) +- * HashAggregate (27) - +- * Expand (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.store (14) - +- BroadcastExchange (23) - +- * ColumnarToRow (22) + +- * ColumnarToRow (26) + +- CometExpand (25) + +- CometProject (24) + +- CometBroadcastHashJoin (23) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.date_dim (9) + : +- CometBroadcastExchange (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.store (15) + +- CometBroadcastExchange (22) +- CometFilter (21) +- CometScan parquet spark_catalog.default.item (20) @@ -43,135 +43,138 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) ColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#10] -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [cd_demo_sk#10] +Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +(8) CometProject Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] + +(9) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#14] +(10) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(11) CometProject +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14], [d_date_sk#14] -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(14) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] +(15) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((isnotnull(s_state#16) AND (s_state#16 = TN)) AND isnotnull(s_store_sk#15)) - -(16) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#16] +(16) CometFilter +Input [2]: [s_store_sk#16, s_state#17] +Condition : ((isnotnull(s_state#17) AND (s_state#17 = TN)) AND isnotnull(s_store_sk#16)) -(17) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_state#17] +Arguments: [s_store_sk#16, s_state#17] -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [2]: [s_store_sk#16, s_state#17] +Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight -(19) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#16] +(19) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#17] +Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] (20) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_item_id#18] +Output [2]: [i_item_sk#18, i_item_id#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (21) CometFilter -Input [2]: [i_item_sk#17, i_item_id#18] -Condition : isnotnull(i_item_sk#17) +Input [2]: [i_item_sk#18, i_item_id#19] +Condition : isnotnull(i_item_sk#18) -(22) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#17, i_item_id#18] +(22) CometBroadcastExchange +Input [2]: [i_item_sk#18, i_item_id#19] +Arguments: [i_item_sk#18, i_item_id#19] -(23) BroadcastExchange -Input [2]: [i_item_sk#17, i_item_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] +Right output [2]: [i_item_sk#18, i_item_id#19] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(24) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#19] +Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17] -(25) Project [codegen id : 5] -Output [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, s_state#16] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16, i_item_sk#17, i_item_id#18] +(25) CometExpand +Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17] +Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#21, spark_grouping_id#22] -(26) Expand [codegen id : 5] -Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, s_state#16] -Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, s_state#16, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#20, spark_grouping_id#21] +(26) ColumnarToRow [codegen id : 1] +Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#21, spark_grouping_id#22] -(27) HashAggregate [codegen id : 5] -Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#20, spark_grouping_id#21] -Keys [3]: [i_item_id#19, s_state#20, spark_grouping_id#21] +(27) HashAggregate [codegen id : 1] +Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#21, spark_grouping_id#22] +Keys [3]: [i_item_id#20, s_state#21, spark_grouping_id#22] Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [8]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Results [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] +Aggregate Attributes [8]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30] +Results [11]: [i_item_id#20, s_state#21, spark_grouping_id#22, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] (28) Exchange -Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] -Arguments: hashpartitioning(i_item_id#19, s_state#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [11]: [i_item_id#20, s_state#21, spark_grouping_id#22, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] +Arguments: hashpartitioning(i_item_id#20, s_state#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 6] -Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] -Keys [3]: [i_item_id#19, s_state#20, spark_grouping_id#21] +(29) HashAggregate [codegen id : 2] +Input [11]: [i_item_id#20, s_state#21, spark_grouping_id#22, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] +Keys [3]: [i_item_id#20, s_state#21, spark_grouping_id#22] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#38, avg(UnscaledValue(ss_list_price#5))#39, avg(UnscaledValue(ss_coupon_amt#7))#40, avg(UnscaledValue(ss_sales_price#6))#41] -Results [7]: [i_item_id#19, s_state#20, cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) AS g_state#42, avg(ss_quantity#4)#38 AS agg1#43, cast((avg(UnscaledValue(ss_list_price#5))#39 / 100.0) as decimal(11,6)) AS agg2#44, cast((avg(UnscaledValue(ss_coupon_amt#7))#40 / 100.0) as decimal(11,6)) AS agg3#45, cast((avg(UnscaledValue(ss_sales_price#6))#41 / 100.0) as decimal(11,6)) AS agg4#46] +Aggregate Attributes [4]: [avg(ss_quantity#4)#39, avg(UnscaledValue(ss_list_price#5))#40, avg(UnscaledValue(ss_coupon_amt#7))#41, avg(UnscaledValue(ss_sales_price#6))#42] +Results [7]: [i_item_id#20, s_state#21, cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint) AS g_state#43, avg(ss_quantity#4)#39 AS agg1#44, cast((avg(UnscaledValue(ss_list_price#5))#40 / 100.0) as decimal(11,6)) AS agg2#45, cast((avg(UnscaledValue(ss_coupon_amt#7))#41 / 100.0) as decimal(11,6)) AS agg3#46, cast((avg(UnscaledValue(ss_sales_price#6))#42 / 100.0) as decimal(11,6)) AS agg4#47] (30) TakeOrderedAndProject -Input [7]: [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] -Arguments: 100, [i_item_id#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST], [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] +Input [7]: [i_item_id#20, s_state#21, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] +Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST], [i_item_id#20, s_state#21, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] ===== Subqueries ===== @@ -184,18 +187,18 @@ BroadcastExchange (35) (31) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#47] +Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (32) CometFilter -Input [2]: [d_date_sk#14, d_year#47] -Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) (33) CometProject -Input [2]: [d_date_sk#14, d_year#47] +Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] (34) ColumnarToRow [codegen id : 1] @@ -203,6 +206,6 @@ Input [1]: [d_date_sk#14] (35) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt index 9d073ff67c..74c8a1ca29 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,s_state,spark_grouping_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] + CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -26,27 +26,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #3 + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #5 + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt index 522754cbc0..1bf9ac34d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt @@ -1,44 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * ColumnarToRow (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometProject (18) + : : : : +- CometBroadcastHashJoin (17) + : : : : :- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (8) + : : : : +- CometBroadcastExchange (16) + : : : : +- CometProject (15) + : : : : +- CometFilter (14) + : : : : +- CometScan parquet spark_catalog.default.date_dim (13) + : : : +- CometBroadcastExchange (22) + : : : +- CometProject (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.date_dim (19) + : : +- CometBroadcastExchange (28) + : : +- CometProject (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- CometBroadcastExchange (33) + : +- CometFilter (32) + : +- CometScan parquet spark_catalog.default.store (31) + +- CometBroadcastExchange (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.item (36) (1) Scan parquet spark_catalog.default.store_sales @@ -53,10 +58,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -(7) BroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] +Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] +(7) CometProject Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -(10) Scan parquet spark_catalog.default.catalog_sales +(8) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] @@ -93,177 +91,183 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] +Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +(12) CometProject Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#19] +(13) Scan parquet spark_catalog.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_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct + +(14) CometFilter +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) + +(15) CometProject +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Arguments: [d_date_sk#19], [d_date_sk#19] -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None +(16) CometBroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: [d_date_sk#19] + +(17) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +(18) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] + +(19) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#22, d_year#23, d_moy#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct -(19) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#20] +(20) CometFilter +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(21) CometProject +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Arguments: [d_date_sk#22], [d_date_sk#22] -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] +(22) CometBroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: [d_date_sk#22] -(22) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#21] +(23) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#22] +Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None +(24) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#22] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(26) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) -(25) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +(27) CometProject +Input [2]: [d_date_sk#25, d_year#26] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(28) CometBroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: [d_date_sk#25] + +(29) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight + +(30) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#25] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] + +(31) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(26) CometFilter -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Condition : isnotnull(s_store_sk#22) - -(27) ColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +(32) CometFilter +Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] +Condition : isnotnull(s_store_sk#27) -(28) BroadcastExchange -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(33) CometBroadcastExchange +Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] +Arguments: [s_store_sk#27, s_store_id#28, s_store_name#29] -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None +(34) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] +Right output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] +Arguments: [ss_store_sk#3], [s_store_sk#27], Inner, BuildRight -(30) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_store_id#23, s_store_name#24] +(35) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#27, s_store_id#28, s_store_name#29] +Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29] -(31) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +(36) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Condition : isnotnull(i_item_sk#25) +(37) CometFilter +Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] +Condition : isnotnull(i_item_sk#30) -(33) ColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +(38) CometBroadcastExchange +Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] +Arguments: [i_item_sk#30, i_item_id#31, i_item_desc#32] -(34) BroadcastExchange -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(39) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29] +Right output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] +Arguments: [ss_item_sk#1], [i_item_sk#30], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#25] -Join type: Inner -Join condition: None +(40) CometProject +Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_sk#30, i_item_id#31, i_item_desc#32] +Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32] -(36) Project [codegen id : 8] -Output [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] -Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24, i_item_sk#25, i_item_id#26, i_item_desc#27] +(41) ColumnarToRow [codegen id : 1] +Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32] -(37) HashAggregate [codegen id : 8] -Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] -Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +(42) HashAggregate [codegen id : 1] +Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32] +Keys [4]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29] Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] -Aggregate Attributes [3]: [sum#28, sum#29, sum#30] -Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] +Aggregate Attributes [3]: [sum#33, sum#34, sum#35] +Results [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum#36, sum#37, sum#38] -(38) Exchange -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(43) Exchange +Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum#36, sum#37, sum#38] +Arguments: hashpartitioning(i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(39) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +(44) HashAggregate [codegen id : 2] +Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum#36, sum#37, sum#38] +Keys [4]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29] Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] -Aggregate Attributes [3]: [sum(ss_quantity#5)#34, sum(sr_return_quantity#11)#35, sum(cs_quantity#16)#36] -Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum(ss_quantity#5)#34 AS store_sales_quantity#37, sum(sr_return_quantity#11)#35 AS store_returns_quantity#38, sum(cs_quantity#16)#36 AS catalog_sales_quantity#39] +Aggregate Attributes [3]: [sum(ss_quantity#5)#39, sum(sr_return_quantity#11)#40, sum(cs_quantity#16)#41] +Results [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum(ss_quantity#5)#39 AS store_sales_quantity#42, sum(sr_return_quantity#11)#40 AS store_returns_quantity#43, sum(cs_quantity#16)#41 AS catalog_sales_quantity#44] -(40) TakeOrderedAndProject -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] -Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] +(45) TakeOrderedAndProject +Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#42, store_returns_quantity#43, catalog_sales_quantity#44] +Arguments: 100, [i_item_id#31 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#42, store_returns_quantity#43, catalog_sales_quantity#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) - - -(41) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#40, d_moy#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] -Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 9)) AND (d_year#40 = 1999)) AND isnotnull(d_date_sk#19)) - -(43) CometProject -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(44) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(45) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (50) +- * ColumnarToRow (49) +- CometProject (48) @@ -272,28 +276,28 @@ BroadcastExchange (50) (46) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#42, d_moy#43] +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_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct (47) CometFilter -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] -Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 9)) AND (d_moy#43 <= 12)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#20)) +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) (48) CometProject -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] -Arguments: [d_date_sk#20], [d_date_sk#20] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Arguments: [d_date_sk#19], [d_date_sk#19] (49) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] +Input [1]: [d_date_sk#19] (50) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 +Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (55) +- * ColumnarToRow (54) +- CometProject (53) @@ -302,25 +306,55 @@ BroadcastExchange (55) (51) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_year#44] +Output [3]: [d_date_sk#22, d_year#23, d_moy#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct (52) CometFilter -Input [2]: [d_date_sk#21, d_year#44] -Condition : (d_year#44 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) (53) CometProject -Input [2]: [d_date_sk#21, d_year#44] -Arguments: [d_date_sk#21], [d_date_sk#21] +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Arguments: [d_date_sk#22], [d_date_sk#22] (54) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#21] +Input [1]: [d_date_sk#22] (55) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 +BroadcastExchange (60) ++- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) + + +(56) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(57) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) + +(58) CometProject +Input [2]: [d_date_sk#25, d_year#26] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(59) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] + +(60) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt index 68a127d357..dbdfc85729 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - WholeStageCodegen (9) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] InputAdapter Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) + WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,53 +31,43 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales CometProject [d_date_sk] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 + CometBroadcastExchange #3 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #7 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #8 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #9 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #10 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange #11 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt index e89cfe1fff..75546d3e55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt @@ -1,25 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (19) ++- * HashAggregate (18) + +- Exchange (17) + +- * HashAggregate (16) + +- * ColumnarToRow (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- CometBroadcastExchange (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.store_sales (4) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.item (9) (1) Scan parquet spark_catalog.default.date_dim @@ -37,10 +35,7 @@ Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) Input [3]: [d_date_sk#1, d_year#2, d_moy#3] Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] -(4) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#1, d_year#2] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -48,78 +43,73 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Condition : isnotnull(ss_item_sk#4) -(7) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [d_date_sk#1, d_year#2] +Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight -(10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +(8) CometProject Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -(11) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) -(13) CometProject +(11) CometProject Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9], [i_item_sk#7, i_brand_id#8, i_brand#9] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9] -(15) BroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(13) CometBroadcastHashJoin +Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#4] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +(14) CometProject Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] + +(15) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(18) HashAggregate [codegen id : 3] +(16) HashAggregate [codegen id : 1] Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -(19) Exchange +(17) Exchange Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum_agg#16] -(21) TakeOrderedAndProject +(19) TakeOrderedAndProject Input [4]: [d_year#2, brand_id#14, brand#15, sum_agg#16] Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, sum_agg#16] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt index 3946c0cd8f..3dc9ede25c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt @@ -1,31 +1,23 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) 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_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_year] CometFilter [d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt index 098d008246..004e046348 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt @@ -1,53 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (33) - : : +- * Filter (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * HashAggregate (28) - : : +- Exchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.web_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : +- BroadcastExchange (39) - : +- * ColumnarToRow (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.customer (36) - +- BroadcastExchange (46) - +- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.customer_address (42) +TakeOrderedAndProject (51) ++- * Project (50) + +- * BroadcastHashJoin Inner BuildRight (49) + :- * Project (43) + : +- * BroadcastHashJoin Inner BuildRight (42) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- Exchange (16) + : : : +- * HashAggregate (15) + : : : +- * ColumnarToRow (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.customer_address (9) + : : +- BroadcastExchange (35) + : : +- * Filter (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * HashAggregate (31) + : : +- * HashAggregate (30) + : : +- Exchange (29) + : : +- * HashAggregate (28) + : : +- * ColumnarToRow (27) + : : +- CometProject (26) + : : +- CometBroadcastHashJoin (25) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.web_returns (19) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : +- BroadcastExchange (41) + : +- * ColumnarToRow (40) + : +- CometFilter (39) + : +- CometScan parquet spark_catalog.default.customer (38) + +- BroadcastExchange (48) + +- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.customer_address (44) (1) Scan parquet spark_catalog.default.web_returns @@ -62,263 +64,272 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 54] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_year#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(6) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] +(7) CometBroadcastHashJoin +Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] +Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -(7) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_state#8] +(9) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#8, ca_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] ReadSchema: struct -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_state#8] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(ca_state#8)) +(10) CometFilter +Input [2]: [ca_address_sk#8, ca_state#9] +Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_state#9)) -(9) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#7, ca_state#8] +(11) CometBroadcastExchange +Input [2]: [ca_address_sk#8, ca_state#9] +Arguments: [ca_address_sk#8, ca_state#9] -(10) BroadcastExchange -Input [2]: [ca_address_sk#7, ca_state#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] +Right output [2]: [ca_address_sk#8, ca_state#9] +Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(13) CometProject +Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#9] +Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -(12) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#8] +(14) ColumnarToRow [codegen id : 1] +Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -(13) HashAggregate [codegen id : 3] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] -Keys [2]: [wr_returning_customer_sk#1, ca_state#8] +(15) HashAggregate [codegen id : 1] +Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] +Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#9] -Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] +Aggregate Attributes [1]: [sum#10] +Results [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] -(14) Exchange -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) Exchange +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 11] -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#8] +(17) HashAggregate [codegen id : 7] +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] +Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#11] -Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#12] +Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#12,17,2) AS ctr_total_return#15] -(16) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) +(18) Filter [codegen id : 7] +Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] +Condition : isnotnull(ctr_total_return#15) -(17) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] +(19) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#18), dynamicpruningexpression(wr_returned_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(wr_returned_date_sk#19), dynamicpruningexpression(wr_returned_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(18) CometFilter -Input [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Condition : isnotnull(wr_returning_addr_sk#16) - -(19) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] - -(20) ReusedExchange [Reuses operator id: 54] -Output [1]: [d_date_sk#20] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#18] -Right keys [1]: [d_date_sk#20] +(20) CometFilter +Input [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] +Condition : isnotnull(wr_returning_addr_sk#17) + +(21) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#21] + +(22) CometBroadcastHashJoin +Left output [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] +Right output [1]: [d_date_sk#21] +Arguments: [wr_returned_date_sk#19], [d_date_sk#21], Inner, BuildRight + +(23) CometProject +Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19, d_date_sk#21] +Arguments: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18], [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18] + +(24) ReusedExchange [Reuses operator id: 11] +Output [2]: [ca_address_sk#22, ca_state#23] + +(25) CometBroadcastHashJoin +Left output [3]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18] +Right output [2]: [ca_address_sk#22, ca_state#23] +Arguments: [wr_returning_addr_sk#17], [ca_address_sk#22], Inner, BuildRight + +(26) CometProject +Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, ca_address_sk#22, ca_state#23] +Arguments: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23], [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23] + +(27) ColumnarToRow [codegen id : 2] +Input [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23] + +(28) HashAggregate [codegen id : 2] +Input [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23] +Keys [2]: [wr_returning_customer_sk#16, ca_state#23] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#18))] +Aggregate Attributes [1]: [sum#24] +Results [3]: [wr_returning_customer_sk#16, ca_state#23, sum#25] + +(29) Exchange +Input [3]: [wr_returning_customer_sk#16, ca_state#23, sum#25] +Arguments: hashpartitioning(wr_returning_customer_sk#16, ca_state#23, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(30) HashAggregate [codegen id : 3] +Input [3]: [wr_returning_customer_sk#16, ca_state#23, sum#25] +Keys [2]: [wr_returning_customer_sk#16, ca_state#23] +Functions [1]: [sum(UnscaledValue(wr_return_amt#18))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#18))#12] +Results [2]: [ca_state#23 AS ctr_state#26, MakeDecimal(sum(UnscaledValue(wr_return_amt#18))#12,17,2) AS ctr_total_return#27] + +(31) HashAggregate [codegen id : 3] +Input [2]: [ctr_state#26, ctr_total_return#27] +Keys [1]: [ctr_state#26] +Functions [1]: [partial_avg(ctr_total_return#27)] +Aggregate Attributes [2]: [sum#28, count#29] +Results [3]: [ctr_state#26, sum#30, count#31] + +(32) Exchange +Input [3]: [ctr_state#26, sum#30, count#31] +Arguments: hashpartitioning(ctr_state#26, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(33) HashAggregate [codegen id : 4] +Input [3]: [ctr_state#26, sum#30, count#31] +Keys [1]: [ctr_state#26] +Functions [1]: [avg(ctr_total_return#27)] +Aggregate Attributes [1]: [avg(ctr_total_return#27)#32] +Results [2]: [(avg(ctr_total_return#27)#32 * 1.2) AS (avg(ctr_total_return) * 1.2)#33, ctr_state#26] + +(34) Filter [codegen id : 4] +Input [2]: [(avg(ctr_total_return) * 1.2)#33, ctr_state#26] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#33) + +(35) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#33, ctr_state#26] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] + +(36) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ctr_state#14] +Right keys [1]: [ctr_state#26] Join type: Inner -Join condition: None +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#33) -(22) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] -Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18, d_date_sk#20] +(37) Project [codegen id : 7] +Output [2]: [ctr_customer_sk#13, ctr_total_return#15] +Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#33, ctr_state#26] -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#21, ca_state#22] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#16] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] -Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, ca_address_sk#21, ca_state#22] - -(26) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] -Keys [2]: [wr_returning_customer_sk#15, ca_state#22] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#17))] -Aggregate Attributes [1]: [sum#23] -Results [3]: [wr_returning_customer_sk#15, ca_state#22, sum#24] - -(27) Exchange -Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#24] -Arguments: hashpartitioning(wr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(28) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#24] -Keys [2]: [wr_returning_customer_sk#15, ca_state#22] -Functions [1]: [sum(UnscaledValue(wr_return_amt#17))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#17))#11] -Results [2]: [ca_state#22 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(wr_return_amt#17))#11,17,2) AS ctr_total_return#26] - -(29) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#25, ctr_total_return#26] -Keys [1]: [ctr_state#25] -Functions [1]: [partial_avg(ctr_total_return#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [3]: [ctr_state#25, sum#29, count#30] - -(30) Exchange -Input [3]: [ctr_state#25, sum#29, count#30] -Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#25, sum#29, count#30] -Keys [1]: [ctr_state#25] -Functions [1]: [avg(ctr_total_return#26)] -Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] -Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(32) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) - -(33) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#13] -Right keys [1]: [ctr_state#25] -Join type: Inner -Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) - -(35) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(36) Scan parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] +(38) Scan parquet spark_catalog.default.customer +Output [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(37) CometFilter -Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] -Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) +(39) CometFilter +Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#36)) -(38) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] +(40) ColumnarToRow [codegen id : 5] +Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] -(39) BroadcastExchange -Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(41) BroadcastExchange +Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#12] -Right keys [1]: [c_customer_sk#33] +(42) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ctr_customer_sk#13] +Right keys [1]: [c_customer_sk#34] Join type: Inner Join condition: None -(41) Project [codegen id : 11] -Output [14]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] -Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] +(43) Project [codegen id : 7] +Output [14]: [ctr_total_return#15, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Input [16]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] -(42) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#47, ca_state#48] +(44) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#48, ca_state#49] 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 -(43) CometFilter -Input [2]: [ca_address_sk#47, ca_state#48] -Condition : ((isnotnull(ca_state#48) AND (ca_state#48 = GA)) AND isnotnull(ca_address_sk#47)) +(45) CometFilter +Input [2]: [ca_address_sk#48, ca_state#49] +Condition : ((isnotnull(ca_state#49) AND (ca_state#49 = GA)) AND isnotnull(ca_address_sk#48)) -(44) CometProject -Input [2]: [ca_address_sk#47, ca_state#48] -Arguments: [ca_address_sk#47], [ca_address_sk#47] +(46) CometProject +Input [2]: [ca_address_sk#48, ca_state#49] +Arguments: [ca_address_sk#48], [ca_address_sk#48] -(45) ColumnarToRow [codegen id : 10] -Input [1]: [ca_address_sk#47] +(47) ColumnarToRow [codegen id : 6] +Input [1]: [ca_address_sk#48] -(46) BroadcastExchange -Input [1]: [ca_address_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(48) BroadcastExchange +Input [1]: [ca_address_sk#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#47] +(49) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#36] +Right keys [1]: [ca_address_sk#48] Join type: Inner Join condition: None -(48) Project [codegen id : 11] -Output [13]: [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46, ctr_total_return#14] -Input [15]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46, ca_address_sk#47] +(50) Project [codegen id : 7] +Output [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#15] +Input [15]: [ctr_total_return#15, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ca_address_sk#48] -(49) TakeOrderedAndProject -Input [13]: [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46, ctr_total_return#14] -Arguments: 100, [c_customer_id#34 ASC NULLS FIRST, c_salutation#36 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, c_last_name#38 ASC NULLS FIRST, c_preferred_cust_flag#39 ASC NULLS FIRST, c_birth_day#40 ASC NULLS FIRST, c_birth_month#41 ASC NULLS FIRST, c_birth_year#42 ASC NULLS FIRST, c_birth_country#43 ASC NULLS FIRST, c_login#44 ASC NULLS FIRST, c_email_address#45 ASC NULLS FIRST, c_last_review_date#46 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46, ctr_total_return#14] +(51) TakeOrderedAndProject +Input [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#15] +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, c_preferred_cust_flag#40 ASC NULLS FIRST, c_birth_day#41 ASC NULLS FIRST, c_birth_month#42 ASC NULLS FIRST, c_birth_year#43 ASC NULLS FIRST, c_birth_country#44 ASC NULLS FIRST, c_login#45 ASC NULLS FIRST, c_email_address#46 ASC NULLS FIRST, c_last_review_date#47 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#15] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +BroadcastExchange (56) ++- * ColumnarToRow (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) -(50) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#49] +(52) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_year#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter -Input [2]: [d_date_sk#6, d_year#49] -Condition : ((isnotnull(d_year#49) AND (d_year#49 = 2002)) AND isnotnull(d_date_sk#6)) +(53) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) -(52) CometProject -Input [2]: [d_date_sk#6, d_year#49] +(54) CometProject +Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(53) ColumnarToRow [codegen id : 1] +(55) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(54) BroadcastExchange +(56) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 17 Hosting Expression = wr_returned_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#19 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt index 365f7f973e..d5b8077ba0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (11) + WholeStageCodegen (7) Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] @@ -10,14 +10,14 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] InputAdapter Exchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -28,52 +28,48 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) + BroadcastExchange #5 + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] InputAdapter - Exchange [ctr_state] #5 - WholeStageCodegen (7) + Exchange [ctr_state] #6 + WholeStageCodegen (3) HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] InputAdapter - Exchange [wr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) + Exchange [wr_returning_customer_sk,ca_state] #7 + WholeStageCodegen (2) HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] CometFilter [wr_returning_addr_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) + BroadcastExchange #8 + WholeStageCodegen (5) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) + BroadcastExchange #9 + WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [ca_address_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt index e20d454864..53dc945327 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt @@ -1,94 +1,99 @@ == Physical Plan == -* Sort (90) -+- Exchange (89) - +- * Project (88) - +- * BroadcastHashJoin Inner BuildRight (87) - :- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * BroadcastHashJoin Inner BuildRight (29) - : : : : :- * HashAggregate (15) - : : : : : +- Exchange (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (6) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- ReusedExchange (4) - : : : : : +- BroadcastExchange (10) - : : : : : +- * ColumnarToRow (9) - : : : : : +- CometFilter (8) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : : : +- BroadcastExchange (28) - : : : : +- * HashAggregate (27) - : : : : +- Exchange (26) - : : : : +- * HashAggregate (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : :- * Project (21) - : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : :- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (16) - : : : : : +- ReusedExchange (19) - : : : : +- ReusedExchange (22) - : : : +- BroadcastExchange (42) - : : : +- * HashAggregate (41) - : : : +- Exchange (40) - : : : +- * HashAggregate (39) - : : : +- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * ColumnarToRow (32) - : : : : : +- CometFilter (31) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (33) - : : : +- ReusedExchange (36) - : : +- BroadcastExchange (57) - : : +- * HashAggregate (56) - : : +- Exchange (55) - : : +- * HashAggregate (54) - : : +- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * ColumnarToRow (47) - : : : : +- CometFilter (46) - : : : : +- CometScan parquet spark_catalog.default.web_sales (45) - : : : +- ReusedExchange (48) - : : +- ReusedExchange (51) - : +- BroadcastExchange (71) - : +- * HashAggregate (70) - : +- Exchange (69) - : +- * HashAggregate (68) - : +- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * ColumnarToRow (61) - : : : +- CometFilter (60) - : : : +- CometScan parquet spark_catalog.default.web_sales (59) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- BroadcastExchange (86) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * BroadcastHashJoin Inner BuildRight (81) - :- * Project (79) - : +- * BroadcastHashJoin Inner BuildRight (78) - : :- * ColumnarToRow (76) - : : +- CometFilter (75) - : : +- CometScan parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (77) - +- ReusedExchange (80) +* Sort (95) ++- Exchange (94) + +- * Project (93) + +- * BroadcastHashJoin Inner BuildRight (92) + :- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : :- * BroadcastHashJoin Inner BuildRight (32) + : : : : :- * HashAggregate (16) + : : : : : +- Exchange (15) + : : : : : +- * HashAggregate (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (8) + : : : : +- BroadcastExchange (31) + : : : : +- * HashAggregate (30) + : : : : +- Exchange (29) + : : : : +- * HashAggregate (28) + : : : : +- * ColumnarToRow (27) + : : : : +- CometProject (26) + : : : : +- CometBroadcastHashJoin (25) + : : : : :- CometProject (23) + : : : : : +- CometBroadcastHashJoin (22) + : : : : : :- CometFilter (18) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (17) + : : : : : +- CometBroadcastExchange (21) + : : : : : +- CometFilter (20) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (19) + : : : : +- ReusedExchange (24) + : : : +- BroadcastExchange (47) + : : : +- * HashAggregate (46) + : : : +- Exchange (45) + : : : +- * HashAggregate (44) + : : : +- * ColumnarToRow (43) + : : : +- CometProject (42) + : : : +- CometBroadcastHashJoin (41) + : : : :- CometProject (39) + : : : : +- CometBroadcastHashJoin (38) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (33) + : : : : +- CometBroadcastExchange (37) + : : : : +- CometFilter (36) + : : : : +- CometScan parquet spark_catalog.default.date_dim (35) + : : : +- ReusedExchange (40) + : : +- BroadcastExchange (62) + : : +- * HashAggregate (61) + : : +- Exchange (60) + : : +- * HashAggregate (59) + : : +- * ColumnarToRow (58) + : : +- CometProject (57) + : : +- CometBroadcastHashJoin (56) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometFilter (51) + : : : : +- CometScan parquet spark_catalog.default.web_sales (50) + : : : +- ReusedExchange (52) + : : +- ReusedExchange (55) + : +- BroadcastExchange (76) + : +- * HashAggregate (75) + : +- Exchange (74) + : +- * HashAggregate (73) + : +- * ColumnarToRow (72) + : +- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometProject (68) + : : +- CometBroadcastHashJoin (67) + : : :- CometFilter (65) + : : : +- CometScan parquet spark_catalog.default.web_sales (64) + : : +- ReusedExchange (66) + : +- ReusedExchange (69) + +- BroadcastExchange (91) + +- * HashAggregate (90) + +- Exchange (89) + +- * HashAggregate (88) + +- * ColumnarToRow (87) + +- CometProject (86) + +- CometBroadcastHashJoin (85) + :- CometProject (83) + : +- CometBroadcastHashJoin (82) + : :- CometFilter (80) + : : +- CometScan parquet spark_catalog.default.web_sales (79) + : +- ReusedExchange (81) + +- ReusedExchange (84) (1) Scan parquet spark_catalog.default.store_sales @@ -103,69 +108,76 @@ ReadSchema: struct Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_addr_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(4) ReusedExchange [Reuses operator id: 94] +(3) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(4) CometFilter +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(6) Project [codegen id : 3] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] +(5) CometBroadcastExchange +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Arguments: [d_date_sk#5, d_year#6, d_qoy#7] + +(6) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Right output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(7) CometProject Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] +Arguments: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7], [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] -(7) Scan parquet spark_catalog.default.customer_address +(8) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#8, ca_county#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] ReadSchema: struct -(8) CometFilter +(9) CometFilter Input [2]: [ca_address_sk#8, ca_county#9] Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) -(9) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [2]: [ca_address_sk#8, ca_county#9] +Arguments: [ca_address_sk#8, ca_county#9] -(10) BroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#8] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] +Right output [2]: [ca_address_sk#8, ca_county#9] +Arguments: [ss_addr_sk#1], [ca_address_sk#8], Inner, BuildRight -(12) Project [codegen id : 3] -Output [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] +(12) CometProject Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] +Arguments: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9], [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] + +(13) ColumnarToRow [codegen id : 1] +Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -(13) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum#10] Results [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -(14) Exchange +(15) Exchange Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 24] +(16) HashAggregate [codegen id : 12] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS store_sales#13] -(16) Scan parquet spark_catalog.default.store_sales +(17) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] @@ -173,68 +185,78 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(17) CometFilter +(18) CometFilter Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] Condition : isnotnull(ss_addr_sk#14) -(18) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] - -(19) ReusedExchange [Reuses operator id: 98] +(19) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#16] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None +(20) CometFilter +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) + +(21) CometBroadcastExchange +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Arguments: [d_date_sk#18, d_year#19, d_qoy#20] + +(22) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] +Right output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Arguments: [ss_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight -(21) Project [codegen id : 6] -Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] +(23) CometProject Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] +Arguments: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20], [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] -(22) ReusedExchange [Reuses operator id: 10] +(24) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#21, ca_county#22] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#14] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] +Right output [2]: [ca_address_sk#21, ca_county#22] +Arguments: [ss_addr_sk#14], [ca_address_sk#21], Inner, BuildRight -(24) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] +(26) CometProject Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] +Arguments: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22], [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -(25) HashAggregate [codegen id : 6] +(27) ColumnarToRow [codegen id : 2] +Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] + +(28) HashAggregate [codegen id : 2] Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] Keys [3]: [ca_county#22, d_qoy#20, d_year#19] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] Aggregate Attributes [1]: [sum#23] Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -(26) Exchange +(29) Exchange Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(27) HashAggregate [codegen id : 7] +(30) HashAggregate [codegen id : 3] Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] Keys [3]: [ca_county#22, d_qoy#20, d_year#19] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#15))#12] Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#12,17,2) AS store_sales#25] -(28) BroadcastExchange +(31) BroadcastExchange Input [2]: [ca_county#22, store_sales#25] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(29) BroadcastHashJoin [codegen id : 24] +(32) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#22] Join type: Inner Join condition: None -(30) Scan parquet spark_catalog.default.store_sales +(33) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] @@ -242,72 +264,82 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(31) CometFilter +(34) CometFilter Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Condition : isnotnull(ss_addr_sk#26) -(32) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] - -(33) ReusedExchange [Reuses operator id: 102] +(35) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None +(36) CometFilter +Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] +Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) + +(37) CometBroadcastExchange +Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] +Arguments: [d_date_sk#30, d_year#31, d_qoy#32] -(35) Project [codegen id : 10] -Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] +(38) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Right output [3]: [d_date_sk#30, d_year#31, d_qoy#32] +Arguments: [ss_sold_date_sk#28], [d_date_sk#30], Inner, BuildRight + +(39) CometProject Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] +Arguments: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32], [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] -(36) ReusedExchange [Reuses operator id: 10] +(40) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#33, ca_county#34] -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#26] -Right keys [1]: [ca_address_sk#33] -Join type: Inner -Join condition: None +(41) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] +Right output [2]: [ca_address_sk#33, ca_county#34] +Arguments: [ss_addr_sk#26], [ca_address_sk#33], Inner, BuildRight -(38) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] +(42) CometProject Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] +Arguments: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34], [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] + +(43) ColumnarToRow [codegen id : 4] +Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -(39) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 4] Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] Keys [3]: [ca_county#34, d_qoy#32, d_year#31] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] Aggregate Attributes [1]: [sum#35] Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -(40) Exchange +(45) Exchange Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(41) HashAggregate [codegen id : 11] +(46) HashAggregate [codegen id : 5] Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] Keys [3]: [ca_county#34, d_qoy#32, d_year#31] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#12] Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#12,17,2) AS store_sales#37] -(42) BroadcastExchange +(47) BroadcastExchange Input [2]: [ca_county#34, store_sales#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 24] +(48) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#22] Right keys [1]: [ca_county#34] Join type: Inner Join condition: None -(44) Project [codegen id : 24] +(49) Project [codegen id : 12] Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] -(45) Scan parquet spark_catalog.default.web_sales +(50) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] @@ -315,68 +347,66 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(46) CometFilter +(51) CometFilter Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Condition : isnotnull(ws_bill_addr_sk#38) -(47) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] - -(48) ReusedExchange [Reuses operator id: 94] +(52) ReusedExchange [Reuses operator id: 5] Output [3]: [d_date_sk#42, d_year#43, d_qoy#44] -(49) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#42] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Right output [3]: [d_date_sk#42, d_year#43, d_qoy#44] +Arguments: [ws_sold_date_sk#40], [d_date_sk#42], Inner, BuildRight -(50) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44] +(54) CometProject Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#42, d_year#43, d_qoy#44] +Arguments: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44], [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44] -(51) ReusedExchange [Reuses operator id: 10] +(55) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#45, ca_county#46] -(52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#45] -Join type: Inner -Join condition: None +(56) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44] +Right output [2]: [ca_address_sk#45, ca_county#46] +Arguments: [ws_bill_addr_sk#38], [ca_address_sk#45], Inner, BuildRight -(53) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] +(57) CometProject Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_address_sk#45, ca_county#46] +Arguments: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46], [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] + +(58) ColumnarToRow [codegen id : 6] +Input [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] -(54) HashAggregate [codegen id : 14] +(59) HashAggregate [codegen id : 6] Input [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] Keys [3]: [ca_county#46, d_qoy#44, d_year#43] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] Aggregate Attributes [1]: [sum#47] Results [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] -(55) Exchange +(60) Exchange Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] -Arguments: hashpartitioning(ca_county#46, d_qoy#44, d_year#43, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(ca_county#46, d_qoy#44, d_year#43, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(56) HashAggregate [codegen id : 15] +(61) HashAggregate [codegen id : 7] Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] Keys [3]: [ca_county#46, d_qoy#44, d_year#43] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#49] Results [2]: [ca_county#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#49,17,2) AS web_sales#50] -(57) BroadcastExchange +(62) BroadcastExchange Input [2]: [ca_county#46, web_sales#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(58) BroadcastHashJoin [codegen id : 24] +(63) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#46] Join type: Inner Join condition: None -(59) Scan parquet spark_catalog.default.web_sales +(64) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] @@ -384,72 +414,70 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(60) CometFilter +(65) CometFilter Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] Condition : isnotnull(ws_bill_addr_sk#51) -(61) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] - -(62) ReusedExchange [Reuses operator id: 98] +(66) ReusedExchange [Reuses operator id: 21] Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] -(63) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#53] -Right keys [1]: [d_date_sk#55] -Join type: Inner -Join condition: None +(67) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] +Right output [3]: [d_date_sk#55, d_year#56, d_qoy#57] +Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight -(64) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57] +(68) CometProject Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56, d_qoy#57] +Arguments: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57], [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57] -(65) ReusedExchange [Reuses operator id: 10] +(69) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#58, ca_county#59] -(66) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#51] -Right keys [1]: [ca_address_sk#58] -Join type: Inner -Join condition: None +(70) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57] +Right output [2]: [ca_address_sk#58, ca_county#59] +Arguments: [ws_bill_addr_sk#51], [ca_address_sk#58], Inner, BuildRight -(67) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] +(71) CometProject Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_address_sk#58, ca_county#59] +Arguments: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59], [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] + +(72) ColumnarToRow [codegen id : 8] +Input [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] -(68) HashAggregate [codegen id : 18] +(73) HashAggregate [codegen id : 8] Input [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] Keys [3]: [ca_county#59, d_qoy#57, d_year#56] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#52))] Aggregate Attributes [1]: [sum#60] Results [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] -(69) Exchange +(74) Exchange Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] -Arguments: hashpartitioning(ca_county#59, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(ca_county#59, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(70) HashAggregate [codegen id : 19] +(75) HashAggregate [codegen id : 9] Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] Keys [3]: [ca_county#59, d_qoy#57, d_year#56] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#52))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#52))#49] Results [2]: [ca_county#59, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#52))#49,17,2) AS web_sales#62] -(71) BroadcastExchange +(76) BroadcastExchange Input [2]: [ca_county#59, web_sales#62] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=9] -(72) BroadcastHashJoin [codegen id : 24] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#46] Right keys [1]: [ca_county#59] Join type: Inner Join condition: (CASE WHEN (web_sales#50 > 0.00) THEN (web_sales#62 / web_sales#50) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) -(73) Project [codegen id : 24] +(78) Project [codegen id : 12] Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62] Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, ca_county#59, web_sales#62] -(74) Scan parquet spark_catalog.default.web_sales +(79) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] @@ -457,160 +485,158 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(75) CometFilter +(80) CometFilter Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] Condition : isnotnull(ws_bill_addr_sk#63) -(76) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] - -(77) ReusedExchange [Reuses operator id: 102] +(81) ReusedExchange [Reuses operator id: 37] Output [3]: [d_date_sk#67, d_year#68, d_qoy#69] -(78) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#67] -Join type: Inner -Join condition: None +(82) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Right output [3]: [d_date_sk#67, d_year#68, d_qoy#69] +Arguments: [ws_sold_date_sk#65], [d_date_sk#67], Inner, BuildRight -(79) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69] +(83) CometProject Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68, d_qoy#69] +Arguments: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69], [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69] -(80) ReusedExchange [Reuses operator id: 10] +(84) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#70, ca_county#71] -(81) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#63] -Right keys [1]: [ca_address_sk#70] -Join type: Inner -Join condition: None +(85) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69] +Right output [2]: [ca_address_sk#70, ca_county#71] +Arguments: [ws_bill_addr_sk#63], [ca_address_sk#70], Inner, BuildRight -(82) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] +(86) CometProject Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_address_sk#70, ca_county#71] +Arguments: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71], [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] -(83) HashAggregate [codegen id : 22] +(87) ColumnarToRow [codegen id : 10] +Input [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] + +(88) HashAggregate [codegen id : 10] Input [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] Keys [3]: [ca_county#71, d_qoy#69, d_year#68] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] Aggregate Attributes [1]: [sum#72] Results [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] -(84) Exchange +(89) Exchange Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] -Arguments: hashpartitioning(ca_county#71, d_qoy#69, d_year#68, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(ca_county#71, d_qoy#69, d_year#68, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(85) HashAggregate [codegen id : 23] +(90) HashAggregate [codegen id : 11] Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] Keys [3]: [ca_county#71, d_qoy#69, d_year#68] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#64))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#64))#49] Results [2]: [ca_county#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#49,17,2) AS web_sales#74] -(86) BroadcastExchange +(91) BroadcastExchange Input [2]: [ca_county#71, web_sales#74] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(87) BroadcastHashJoin [codegen id : 24] +(92) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#46] Right keys [1]: [ca_county#71] Join type: Inner Join condition: (CASE WHEN (web_sales#62 > 0.00) THEN (web_sales#74 / web_sales#62) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) -(88) Project [codegen id : 24] +(93) Project [codegen id : 12] Output [6]: [ca_county#9, d_year#6, (web_sales#62 / web_sales#50) AS web_q1_q2_increase#75, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#76, (web_sales#74 / web_sales#62) AS web_q2_q3_increase#77, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#78] Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62, ca_county#71, web_sales#74] -(89) Exchange +(94) Exchange Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(90) Sort [codegen id : 25] +(95) Sort [codegen id : 13] Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] Arguments: [ca_county#9 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (94) -+- * ColumnarToRow (93) - +- CometFilter (92) - +- CometScan parquet spark_catalog.default.date_dim (91) +BroadcastExchange (99) ++- * ColumnarToRow (98) + +- CometFilter (97) + +- CometScan parquet spark_catalog.default.date_dim (96) -(91) Scan parquet spark_catalog.default.date_dim +(96) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(92) CometFilter +(97) CometFilter Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) ColumnarToRow [codegen id : 1] +(98) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(94) BroadcastExchange +(99) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (98) -+- * ColumnarToRow (97) - +- CometFilter (96) - +- CometScan parquet spark_catalog.default.date_dim (95) +Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 +BroadcastExchange (103) ++- * ColumnarToRow (102) + +- CometFilter (101) + +- CometScan parquet spark_catalog.default.date_dim (100) -(95) Scan parquet spark_catalog.default.date_dim +(100) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(96) CometFilter +(101) CometFilter Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) -(97) ColumnarToRow [codegen id : 1] +(102) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -(98) BroadcastExchange +(103) BroadcastExchange Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] -Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 -BroadcastExchange (102) -+- * ColumnarToRow (101) - +- CometFilter (100) - +- CometScan parquet spark_catalog.default.date_dim (99) +Subquery:3 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 +BroadcastExchange (107) ++- * ColumnarToRow (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) -(99) Scan parquet spark_catalog.default.date_dim +(104) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(100) CometFilter +(105) CometFilter Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) -(101) ColumnarToRow [codegen id : 1] +(106) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -(102) BroadcastExchange +(107) BroadcastExchange Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -Subquery:4 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 59 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#17 +Subquery:5 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#17 -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#29 +Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#29 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt index f4bf6a89d0..c004db9b75 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (25) +WholeStageCodegen (13) Sort [ca_county] InputAdapter Exchange [ca_county] #1 - WholeStageCodegen (24) + WholeStageCodegen (12) Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] @@ -14,14 +14,14 @@ WholeStageCodegen (25) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,129 +31,120 @@ WholeStageCodegen (25) InputAdapter CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange #4 + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #5 + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (3) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) + Exchange [ca_county,d_qoy,d_year] #7 + WholeStageCodegen (2) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + CometBroadcastExchange #9 + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) + Exchange [ca_county,d_qoy,d_year] #11 + WholeStageCodegen (4) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 + BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + CometBroadcastExchange #13 + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #14 + WholeStageCodegen (7) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) + Exchange [ca_county,d_qoy,d_year] #15 + WholeStageCodegen (6) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #4 + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) + BroadcastExchange #16 + WholeStageCodegen (9) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #14 - WholeStageCodegen (18) + Exchange [ca_county,d_qoy,d_year] #17 + WholeStageCodegen (8) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #9 + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) + BroadcastExchange #18 + WholeStageCodegen (11) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #16 - WholeStageCodegen (22) + Exchange [ca_county,d_qoy,d_year] #19 + WholeStageCodegen (10) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #13 + ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt index bd7caaaa11..fe9e7d49a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt @@ -1,33 +1,35 @@ == Physical Plan == -* HashAggregate (29) -+- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * HashAggregate (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * ColumnarToRow (13) - : : +- CometFilter (12) - : : +- CometScan parquet spark_catalog.default.catalog_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (24) +* HashAggregate (31) ++- Exchange (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * ColumnarToRow (9) + : : +- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.item (3) + : +- BroadcastExchange (23) + : +- * Filter (22) + : +- * HashAggregate (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- * ColumnarToRow (18) + : +- CometProject (17) + : +- CometBroadcastHashJoin (16) + : :- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.catalog_sales (10) + : +- CometBroadcastExchange (15) + : +- CometProject (14) + : +- CometFilter (13) + : +- CometScan parquet spark_catalog.default.date_dim (12) + +- ReusedExchange (26) (1) Scan parquet spark_catalog.default.catalog_sales @@ -42,42 +44,38 @@ ReadSchema: struct Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) -(3) ColumnarToRow [codegen id : 6] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#5, i_manufact_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [i_item_sk#5, i_manufact_id#6] Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) -(6) CometProject +(5) CometProject Input [2]: [i_item_sk#5, i_manufact_id#6] Arguments: [i_item_sk#5], [i_item_sk#5] -(7) ColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [1]: [i_item_sk#5] +Arguments: [i_item_sk#5] -(8) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Right output [1]: [i_item_sk#5] +Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] +(8) CometProject Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] +Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5], [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] + +(9) ColumnarToRow [codegen id : 4] +Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -(11) Scan parquet spark_catalog.default.catalog_sales +(10) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -85,125 +83,140 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(11) CometFilter Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] Condition : isnotnull(cs_item_sk#7) -(13) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] +(12) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#11] +(13) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] -(16) Project [codegen id : 3] -Output [2]: [cs_item_sk#7, cs_ext_discount_amt#8] +(15) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(16) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] +Right output [1]: [d_date_sk#11] +Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight + +(17) CometProject Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#11] +Arguments: [cs_item_sk#7, cs_ext_discount_amt#8], [cs_item_sk#7, cs_ext_discount_amt#8] + +(18) ColumnarToRow [codegen id : 1] +Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -(17) HashAggregate [codegen id : 3] +(19) HashAggregate [codegen id : 1] Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] Keys [1]: [cs_item_sk#7] Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] -Aggregate Attributes [2]: [sum#12, count#13] -Results [3]: [cs_item_sk#7, sum#14, count#15] +Aggregate Attributes [2]: [sum#13, count#14] +Results [3]: [cs_item_sk#7, sum#15, count#16] -(18) Exchange -Input [3]: [cs_item_sk#7, sum#14, count#15] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(20) Exchange +Input [3]: [cs_item_sk#7, sum#15, count#16] +Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(19) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#7, sum#14, count#15] +(21) HashAggregate [codegen id : 2] +Input [3]: [cs_item_sk#7, sum#15, count#16] Keys [1]: [cs_item_sk#7] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#16] -Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#17] +Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#17 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#7] -(20) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) +(22) Filter [codegen id : 2] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#7] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#18) -(21) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] +(23) BroadcastExchange +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=2] -(22) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#5] Right keys [1]: [cs_item_sk#7] Join type: Inner -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#18) -(23) Project [codegen id : 6] +(25) Project [codegen id : 4] Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#18, cs_item_sk#7] -(24) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#18] +(26) ReusedExchange [Reuses operator id: 36] +Output [1]: [d_date_sk#19] -(25) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [1]: [cs_ext_discount_amt#2] -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#18] +Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#19] -(27) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 4] Input [1]: [cs_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#19] -Results [1]: [sum#20] +Aggregate Attributes [1]: [sum#20] +Results [1]: [sum#21] -(28) Exchange -Input [1]: [sum#20] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(30) Exchange +Input [1]: [sum#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(29) HashAggregate [codegen id : 7] -Input [1]: [sum#20] +(31) HashAggregate [codegen id : 5] +Input [1]: [sum#21] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#21] -Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#21,17,2) AS excess discount amount#22] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#22] +Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#22,17,2) AS excess discount amount#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (34) -+- * ColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.date_dim (30) +BroadcastExchange (36) ++- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(30) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#23] +(32) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#19, d_date#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [d_date_sk#18, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) +(33) CometFilter +Input [2]: [d_date_sk#19, d_date#24] +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 2000-01-27)) AND (d_date#24 <= 2000-04-26)) AND isnotnull(d_date_sk#19)) -(32) CometProject -Input [2]: [d_date_sk#18, d_date#23] -Arguments: [d_date_sk#18], [d_date_sk#18] +(34) CometProject +Input [2]: [d_date_sk#19, d_date#24] +Arguments: [d_date_sk#19], [d_date_sk#19] -(33) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] +(35) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#19] -(34) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) BroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt index 146a33fdd2..8cc26d8f41 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt @@ -1,17 +1,17 @@ -WholeStageCodegen (7) +WholeStageCodegen (5) HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] InputAdapter Exchange #1 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cs_ext_discount_amt] [sum,sum] Project [cs_ext_discount_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_ext_discount_amt,cs_sold_date_sk] BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] - Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk,cs_ext_discount_amt] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -22,31 +22,29 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange #3 + CometProject [i_item_sk] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Filter [(1.3 * avg(cs_ext_discount_amt))] HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] InputAdapter Exchange [cs_item_sk] #5 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] - Project [cs_item_sk,cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt index f9541481cd..4c00423c02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt @@ -7,61 +7,61 @@ TakeOrderedAndProject (63) :- * HashAggregate (28) : +- Exchange (27) : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) + : +- * ColumnarToRow (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.customer_address (9) + : +- CometBroadcastExchange (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) : +- CometProject (19) : +- CometFilter (18) : +- CometScan parquet spark_catalog.default.item (17) :- * HashAggregate (43) : +- Exchange (42) : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * ColumnarToRow (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) + : +- * ColumnarToRow (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) +- * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * ColumnarToRow (46) - : : : +- CometFilter (45) - : : : +- CometScan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) + +- * ColumnarToRow (55) + +- CometProject (54) + +- CometBroadcastHashJoin (53) + :- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometProject (48) + : : +- CometBroadcastHashJoin (47) + : : :- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet spark_catalog.default.store_sales @@ -76,295 +76,292 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(6) Project [codegen id : 5] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +(7) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -(7) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] +(9) Scan parquet spark_catalog.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 -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] +(10) CometFilter +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)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] +(11) CometProject +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] +Arguments: [ca_address_sk#9], [ca_address_sk#9] -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [ca_address_sk#9] +Arguments: [ca_address_sk#9] -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +Right output [1]: [ca_address_sk#9] +Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight -(13) Project [codegen id : 5] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] +(14) CometProject +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] +Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] -(14) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_manufact_id#10] +(15) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [i_item_sk#9, i_manufact_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#9, i_manufact_id#10] +(16) CometFilter +Input [2]: [i_item_sk#11, i_manufact_id#12] +Condition : isnotnull(i_item_sk#11) (17) Scan parquet spark_catalog.default.item -Output [2]: [i_category#11, i_manufact_id#12] +Output [2]: [i_category#13, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (18) CometFilter -Input [2]: [i_category#11, i_manufact_id#12] -Condition : (isnotnull(i_category#11) AND (i_category#11 = Electronics )) +Input [2]: [i_category#13, i_manufact_id#14] +Condition : (isnotnull(i_category#13) AND (i_category#13 = Electronics )) (19) CometProject -Input [2]: [i_category#11, i_manufact_id#12] -Arguments: [i_manufact_id#12], [i_manufact_id#12] - -(20) ColumnarToRow [codegen id : 3] -Input [1]: [i_manufact_id#12] - -(21) BroadcastExchange -Input [1]: [i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_manufact_id#10] -Right keys [1]: [i_manufact_id#12] -Join type: LeftSemi -Join condition: None - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_manufact_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#3, i_manufact_id#10] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_manufact_id#10] - -(26) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#3, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_category#13, i_manufact_id#14] +Arguments: [i_manufact_id#14], [i_manufact_id#14] + +(20) CometBroadcastExchange +Input [1]: [i_manufact_id#14] +Arguments: [i_manufact_id#14] + +(21) CometBroadcastHashJoin +Left output [2]: [i_item_sk#11, i_manufact_id#12] +Right output [1]: [i_manufact_id#14] +Arguments: [i_manufact_id#12], [i_manufact_id#14], LeftSemi, BuildRight + +(22) CometBroadcastExchange +Input [2]: [i_item_sk#11, i_manufact_id#12] +Arguments: [i_item_sk#11, i_manufact_id#12] + +(23) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] +Right output [2]: [i_item_sk#11, i_manufact_id#12] +Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight + +(24) CometProject +Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_manufact_id#12] +Arguments: [ss_ext_sales_price#3, i_manufact_id#12], [ss_ext_sales_price#3, i_manufact_id#12] + +(25) ColumnarToRow [codegen id : 1] +Input [2]: [ss_ext_sales_price#3, i_manufact_id#12] + +(26) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_sales_price#3, i_manufact_id#12] +Keys [1]: [i_manufact_id#12] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [i_manufact_id#10, sum#14] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_manufact_id#12, sum#16] (27) Exchange -Input [2]: [i_manufact_id#10, sum#14] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [2]: [i_manufact_id#12, sum#16] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 6] -Input [2]: [i_manufact_id#10, sum#14] -Keys [1]: [i_manufact_id#10] +(28) HashAggregate [codegen id : 2] +Input [2]: [i_manufact_id#12, sum#16] +Keys [1]: [i_manufact_id#12] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#17] +Results [2]: [i_manufact_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS total_sales#18] (29) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Output [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_sold_date_sk#22 IN dynamicpruning#23)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (30) CometFilter -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) - -(31) ColumnarToRow [codegen id : 11] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] - -(32) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#22] - -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] - -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#23] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#23] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] - -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#24, i_manufact_id#25] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#24] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#19, i_manufact_id#25] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_manufact_id#25] - -(41) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#19, i_manufact_id#25] -Keys [1]: [i_manufact_id#25] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_manufact_id#25, sum#27] +Input [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22] +Condition : (isnotnull(cs_bill_addr_sk#19) AND isnotnull(cs_item_sk#20)) + +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#24] + +(32) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight + +(33) CometProject +Input [5]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22, d_date_sk#24] +Arguments: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21], [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21] + +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#25] + +(35) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21] +Right output [1]: [ca_address_sk#25] +Arguments: [cs_bill_addr_sk#19], [ca_address_sk#25], Inner, BuildRight + +(36) CometProject +Input [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, ca_address_sk#25] +Arguments: [cs_item_sk#20, cs_ext_sales_price#21], [cs_item_sk#20, cs_ext_sales_price#21] + +(37) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#26, i_manufact_id#27] + +(38) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#20, cs_ext_sales_price#21] +Right output [2]: [i_item_sk#26, i_manufact_id#27] +Arguments: [cs_item_sk#20], [i_item_sk#26], Inner, BuildRight + +(39) CometProject +Input [4]: [cs_item_sk#20, cs_ext_sales_price#21, i_item_sk#26, i_manufact_id#27] +Arguments: [cs_ext_sales_price#21, i_manufact_id#27], [cs_ext_sales_price#21, i_manufact_id#27] + +(40) ColumnarToRow [codegen id : 3] +Input [2]: [cs_ext_sales_price#21, i_manufact_id#27] + +(41) HashAggregate [codegen id : 3] +Input [2]: [cs_ext_sales_price#21, i_manufact_id#27] +Keys [1]: [i_manufact_id#27] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#21))] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_manufact_id#27, sum#29] (42) Exchange -Input [2]: [i_manufact_id#25, sum#27] -Arguments: hashpartitioning(i_manufact_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [2]: [i_manufact_id#27, sum#29] +Arguments: hashpartitioning(i_manufact_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#25, sum#27] -Keys [1]: [i_manufact_id#25] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] -Results [2]: [i_manufact_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] +(43) HashAggregate [codegen id : 4] +Input [2]: [i_manufact_id#27, sum#29] +Keys [1]: [i_manufact_id#27] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#21))#30] +Results [2]: [i_manufact_id#27, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#21))#30,17,2) AS total_sales#31] (44) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (45) CometFilter -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) - -(46) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] - -(47) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#35] - -(48) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#35] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 17] -Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] - -(50) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#36] - -(51) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#31] -Right keys [1]: [ca_address_sk#36] -Join type: Inner -Join condition: None - -(52) Project [codegen id : 17] -Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] - -(53) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#37, i_manufact_id#38] - -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#30] -Right keys [1]: [i_item_sk#37] -Join type: Inner -Join condition: None - -(55) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#32, i_manufact_id#38] -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_manufact_id#38] - -(56) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#32, i_manufact_id#38] -Keys [1]: [i_manufact_id#38] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum#39] -Results [2]: [i_manufact_id#38, sum#40] +Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] +Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) + +(46) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#37] + +(47) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight + +(48) CometProject +Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#37] +Arguments: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34], [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] + +(49) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#38] + +(50) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] +Right output [1]: [ca_address_sk#38] +Arguments: [ws_bill_addr_sk#33], [ca_address_sk#38], Inner, BuildRight + +(51) CometProject +Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#38] +Arguments: [ws_item_sk#32, ws_ext_sales_price#34], [ws_item_sk#32, ws_ext_sales_price#34] + +(52) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#39, i_manufact_id#40] + +(53) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#32, ws_ext_sales_price#34] +Right output [2]: [i_item_sk#39, i_manufact_id#40] +Arguments: [ws_item_sk#32], [i_item_sk#39], Inner, BuildRight + +(54) CometProject +Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#39, i_manufact_id#40] +Arguments: [ws_ext_sales_price#34, i_manufact_id#40], [ws_ext_sales_price#34, i_manufact_id#40] + +(55) ColumnarToRow [codegen id : 5] +Input [2]: [ws_ext_sales_price#34, i_manufact_id#40] + +(56) HashAggregate [codegen id : 5] +Input [2]: [ws_ext_sales_price#34, i_manufact_id#40] +Keys [1]: [i_manufact_id#40] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] +Aggregate Attributes [1]: [sum#41] +Results [2]: [i_manufact_id#40, sum#42] (57) Exchange -Input [2]: [i_manufact_id#38, sum#40] -Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [2]: [i_manufact_id#40, sum#42] +Arguments: hashpartitioning(i_manufact_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(58) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#38, sum#40] -Keys [1]: [i_manufact_id#38] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] -Results [2]: [i_manufact_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] +(58) HashAggregate [codegen id : 6] +Input [2]: [i_manufact_id#40, sum#42] +Keys [1]: [i_manufact_id#40] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#34))#43] +Results [2]: [i_manufact_id#40, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34))#43,17,2) AS total_sales#44] (59) Union -(60) HashAggregate [codegen id : 19] -Input [2]: [i_manufact_id#10, total_sales#16] -Keys [1]: [i_manufact_id#10] -Functions [1]: [partial_sum(total_sales#16)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [3]: [i_manufact_id#10, sum#45, isEmpty#46] +(60) HashAggregate [codegen id : 7] +Input [2]: [i_manufact_id#12, total_sales#18] +Keys [1]: [i_manufact_id#12] +Functions [1]: [partial_sum(total_sales#18)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [3]: [i_manufact_id#12, sum#47, isEmpty#48] (61) Exchange -Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [3]: [i_manufact_id#12, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(62) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] -Keys [1]: [i_manufact_id#10] -Functions [1]: [sum(total_sales#16)] -Aggregate Attributes [1]: [sum(total_sales#16)#47] -Results [2]: [i_manufact_id#10, sum(total_sales#16)#47 AS total_sales#48] +(62) HashAggregate [codegen id : 8] +Input [3]: [i_manufact_id#12, sum#47, isEmpty#48] +Keys [1]: [i_manufact_id#12] +Functions [1]: [sum(total_sales#18)] +Aggregate Attributes [1]: [sum(total_sales#18)#49] +Results [2]: [i_manufact_id#12, sum(total_sales#18)#49 AS total_sales#50] (63) TakeOrderedAndProject -Input [2]: [i_manufact_id#10, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#10, total_sales#48] +Input [2]: [i_manufact_id#12, total_sales#50] +Arguments: 100, [total_sales#50 ASC NULLS FIRST], [i_manufact_id#12, total_sales#50] ===== Subqueries ===== @@ -377,18 +374,18 @@ BroadcastExchange (68) (64) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#49, d_moy#50] +Output [3]: [d_date_sk#6, d_year#7, d_moy#8] 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,5), IsNotNull(d_date_sk)] ReadSchema: struct (65) CometFilter -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 5)) AND isnotnull(d_date_sk#6)) +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) (66) CometProject -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] (67) ColumnarToRow [codegen id : 1] @@ -396,10 +393,10 @@ Input [1]: [d_date_sk#6] (68) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#22 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt index 4ab82379ff..84143e0153 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (20) + WholeStageCodegen (8) HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_manufact_id] #1 - WholeStageCodegen (19) + WholeStageCodegen (7) HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_manufact_id] #2 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_manufact_id] - 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] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,75 +31,59 @@ TakeOrderedAndProject [total_sales,i_manufact_id] CometProject [d_date_sk] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_manufact_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (12) + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange #6 + CometBroadcastHashJoin [i_manufact_id,i_manufact_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange #7 + CometProject [i_manufact_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] + WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #7 - WholeStageCodegen (11) + Exchange [i_manufact_id] #8 + WholeStageCodegen (3) HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_manufact_id] - 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] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 - WholeStageCodegen (18) + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 + WholeStageCodegen (6) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #8 - WholeStageCodegen (17) + Exchange [i_manufact_id] #9 + WholeStageCodegen (5) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_manufact_id] - 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] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt index d0f166fe9d..5609dc11c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt @@ -1,36 +1,37 @@ == Physical Plan == -* Sort (32) -+- Exchange (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (24) - : +- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (28) - +- * ColumnarToRow (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.customer (25) +* Sort (33) ++- Exchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Filter (25) + : +- * HashAggregate (24) + : +- Exchange (23) + : +- * ColumnarToRow (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.household_demographics (15) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.customer (26) (1) Scan parquet spark_catalog.default.store_sales @@ -45,174 +46,179 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#7] +(4) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(7) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +(8) CometProject Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#10, s_county#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] +(10) CometFilter +Input [2]: [s_store_sk#10, s_county#11] +Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] +(11) CometProject +Input [2]: [s_store_sk#10, s_county#11] +Arguments: [s_store_sk#10], [s_store_sk#10] -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#10] +Arguments: [s_store_sk#10] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +Right output [1]: [s_store_sk#10] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] +(14) CometProject +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -(14) Scan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +(15) Scan parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) +(16) CometFilter +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Condition : ((((isnotnull(hd_vehicle_count#15) AND ((hd_buy_potential#13 = >10000 ) OR (hd_buy_potential#13 = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN ((cast(hd_dep_count#14 as double) / cast(hd_vehicle_count#15 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#12)) -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] +(17) CometProject +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Arguments: [hd_demo_sk#12], [hd_demo_sk#12] -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#12] +Arguments: [hd_demo_sk#12] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(19) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Right output [1]: [hd_demo_sk#12] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None +(20) CometProject +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] +Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] +(21) CometHashAggregate Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) Exchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) ColumnarToRow [codegen id : 1] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] + +(23) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(23) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +(24) HashAggregate [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] +Aggregate Attributes [1]: [count(1)#17] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(24) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) +(25) Filter [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18] +Condition : ((cnt#18 >= 15) AND (cnt#18 <= 20)) -(25) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(26) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) +(27) CometFilter +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Condition : isnotnull(c_customer_sk#19) -(27) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(28) ColumnarToRow [codegen id : 2] +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(28) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(29) BroadcastExchange +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(29) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#19] Join type: Inner Join condition: None -(30) Project [codegen id : 6] -Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(31) Project [codegen id : 3] +Output [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18, c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(31) Exchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(32) Exchange +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: rangepartitioning(c_last_name#22 ASC NULLS FIRST, c_first_name#21 ASC NULLS FIRST, c_salutation#20 ASC NULLS FIRST, c_preferred_cust_flag#23 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 7] -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST], true, 0 +(33) Sort [codegen id : 4] +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: [c_last_name#22 ASC NULLS FIRST, c_first_name#21 ASC NULLS FIRST, c_salutation#20 ASC NULLS FIRST, c_preferred_cust_flag#23 DESC NULLS LAST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(33) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +(34) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_dom#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] -Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(35) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(35) CometProject -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +(36) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(37) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt index 80405a784d..eefd383438 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt @@ -1,24 +1,24 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] InputAdapter Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [cnt] HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter Exchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,27 +29,21 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange #5 + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange #6 + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt index a46018cdd0..c06c1dd16f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (28) + : : +- * Filter (27) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) + : : : :- * ColumnarToRow (12) + : : : : +- CometBroadcastHashJoin (11) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (10) + : : : : +- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) Scan parquet spark_catalog.default.customer @@ -54,228 +56,237 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#9] +(5) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(6) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(7) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] + +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] +(9) CometProject Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ss_customer_sk#6], [ss_customer_sk#6] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#6] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Right output [1]: [ss_customer_sk#6] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight + +(12) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(13) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#15] -(13) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#13] +(15) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +Right output [1]: [d_date_sk#15] +Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(16) CometProject +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] +Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +(17) ColumnarToRow [codegen id : 1] +Input [1]: [ws_bill_customer_sk#12] -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) BroadcastExchange +Input [1]: [ws_bill_customer_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 9] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] +Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#19] -(20) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#17] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(23) CometProject +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#14] -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] +(24) ColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#16] -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#14] +Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(27) Filter [codegen id : 5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(28) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(27) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +(29) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) +(30) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(29) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] +(31) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#20, ca_state#21] -(30) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(32) BroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(31) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(32) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#19] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#19] +(34) Project [codegen id : 5] +Output [2]: [c_current_cdemo_sk#4, ca_state#21] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(33) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(35) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(34) CometFilter -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Condition : isnotnull(cd_demo_sk#20) +(36) CometFilter +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Condition : isnotnull(cd_demo_sk#22) -(35) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(37) ColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(36) BroadcastExchange -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(38) BroadcastExchange +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#20] +Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 9] -Output [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Input [8]: [c_current_cdemo_sk#4, ca_state#19, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(40) Project [codegen id : 5] +Output [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(39) HashAggregate [codegen id : 9] -Input [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#23), partial_max(cd_dep_count#23), partial_avg(cd_dep_count#23), partial_min(cd_dep_employed_count#24), partial_max(cd_dep_employed_count#24), partial_avg(cd_dep_employed_count#24), partial_min(cd_dep_college_count#25), partial_max(cd_dep_college_count#25), partial_avg(cd_dep_college_count#25)] -Aggregate Attributes [13]: [count#26, min#27, max#28, sum#29, count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38] -Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] +(41) HashAggregate [codegen id : 5] +Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#25), partial_max(cd_dep_count#25), partial_avg(cd_dep_count#25), partial_min(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_avg(cd_dep_employed_count#26), partial_min(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_avg(cd_dep_college_count#27)] +Aggregate Attributes [13]: [count#28, min#29, max#30, sum#31, count#32, min#33, max#34, sum#35, count#36, min#37, max#38, sum#39, count#40] +Results [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53] -(40) Exchange -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(42) Exchange +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53] +Arguments: hashpartitioning(ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(41) HashAggregate [codegen id : 10] -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [count(1), min(cd_dep_count#23), max(cd_dep_count#23), avg(cd_dep_count#23), min(cd_dep_employed_count#24), max(cd_dep_employed_count#24), avg(cd_dep_employed_count#24), min(cd_dep_college_count#25), max(cd_dep_college_count#25), avg(cd_dep_college_count#25)] -Aggregate Attributes [10]: [count(1)#52, min(cd_dep_count#23)#53, max(cd_dep_count#23)#54, avg(cd_dep_count#23)#55, min(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, avg(cd_dep_employed_count#24)#58, min(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, avg(cd_dep_college_count#25)#61] -Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, count(1)#52 AS cnt1#62, min(cd_dep_count#23)#53 AS min(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, avg(cd_dep_count#23)#55 AS avg(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, min(cd_dep_employed_count#24)#56 AS min(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, avg(cd_dep_employed_count#24)#58 AS avg(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, min(cd_dep_college_count#25)#59 AS min(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, avg(cd_dep_college_count#25)#61 AS avg(cd_dep_college_count)#73, cd_dep_count#23] +(43) HashAggregate [codegen id : 6] +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [count(1), min(cd_dep_count#25), max(cd_dep_count#25), avg(cd_dep_count#25), min(cd_dep_employed_count#26), max(cd_dep_employed_count#26), avg(cd_dep_employed_count#26), min(cd_dep_college_count#27), max(cd_dep_college_count#27), avg(cd_dep_college_count#27)] +Aggregate Attributes [10]: [count(1)#54, min(cd_dep_count#25)#55, max(cd_dep_count#25)#56, avg(cd_dep_count#25)#57, min(cd_dep_employed_count#26)#58, max(cd_dep_employed_count#26)#59, avg(cd_dep_employed_count#26)#60, min(cd_dep_college_count#27)#61, max(cd_dep_college_count#27)#62, avg(cd_dep_college_count#27)#63] +Results [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, count(1)#54 AS cnt1#64, min(cd_dep_count#25)#55 AS min(cd_dep_count)#65, max(cd_dep_count#25)#56 AS max(cd_dep_count)#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, cd_dep_employed_count#26, count(1)#54 AS cnt2#68, min(cd_dep_employed_count#26)#58 AS min(cd_dep_employed_count)#69, max(cd_dep_employed_count#26)#59 AS max(cd_dep_employed_count)#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, cd_dep_college_count#27, count(1)#54 AS cnt3#72, min(cd_dep_college_count#27)#61 AS min(cd_dep_college_count)#73, max(cd_dep_college_count#27)#62 AS max(cd_dep_college_count)#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, cd_dep_count#25] -(42) TakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73, cd_dep_count#23] -Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73] +(44) TakeOrderedAndProject +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cnt1#64, min(cd_dep_count)#65, max(cd_dep_count)#66, avg(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, min(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, avg(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, min(cd_dep_college_count)#73, max(cd_dep_college_count)#74, avg(cd_dep_college_count)#75, cd_dep_count#25] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#23, cd_marital_status#24, cnt1#64, min(cd_dep_count)#65, max(cd_dep_count)#66, avg(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, min(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, avg(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, min(cd_dep_college_count)#73, max(cd_dep_college_count)#74, avg(cd_dep_college_count)#75] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(43) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#74, d_qoy#75] +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] -Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) +(46) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(45) CometProject -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +(47) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt index ea0ef274ea..efe0b0b4ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (6) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,61 +13,57 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Filter [exists,exists] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt index 39b8381577..1fdba8689b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt @@ -1,32 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * Project (27) - +- Window (26) - +- * Sort (25) - +- Exchange (24) - +- * HashAggregate (23) - +- Exchange (22) - +- * HashAggregate (21) - +- * Expand (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store (13) +TakeOrderedAndProject (29) ++- * Project (28) + +- Window (27) + +- * Sort (26) + +- Exchange (25) + +- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * ColumnarToRow (21) + +- CometExpand (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.item (9) + +- CometBroadcastExchange (17) + +- CometProject (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) (1) Scan parquet spark_catalog.default.store_sales @@ -41,154 +42,161 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 33] -Output [1]: [d_date_sk#7] +(4) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#7, d_year#8] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight + +(8) CometProject Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -(7) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] +(9) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#9, i_class#10, i_category#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) - -(9) ColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#9, i_category#10] +(10) CometFilter +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Condition : isnotnull(i_item_sk#9) -(10) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Arguments: [i_item_sk#9, i_class#10, i_category#11] -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +Right output [3]: [i_item_sk#9, i_class#10, i_category#11] +Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight -(12) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#9, i_category#10] +(13) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#10, i_category#11] +Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_state#12] +(14) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#12, s_state#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [2]: [s_store_sk#11, s_state#12] -Condition : ((isnotnull(s_state#12) AND (s_state#12 = TN)) AND isnotnull(s_store_sk#11)) +(15) CometFilter +Input [2]: [s_store_sk#12, s_state#13] +Condition : ((isnotnull(s_state#13) AND (s_state#13 = TN)) AND isnotnull(s_store_sk#12)) -(15) CometProject -Input [2]: [s_store_sk#11, s_state#12] -Arguments: [s_store_sk#11], [s_store_sk#11] +(16) CometProject +Input [2]: [s_store_sk#12, s_state#13] +Arguments: [s_store_sk#12], [s_store_sk#12] -(16) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#11] +(17) CometBroadcastExchange +Input [1]: [s_store_sk#12] +Arguments: [s_store_sk#12] -(17) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastHashJoin +Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] +Right output [1]: [s_store_sk#12] +Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(19) CometProject +Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11, s_store_sk#12] +Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_category#11, i_class#10], [ss_ext_sales_price#3, ss_net_profit#4, i_category#11, i_class#10] -(19) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#10, i_class#9] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10, s_store_sk#11] +(20) CometExpand +Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#11, i_class#10] +Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#11, i_class#10, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#11, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#14, i_class#15, spark_grouping_id#16] -(20) Expand [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#10, i_class#9] -Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#10, i_class#9, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#10, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#14, spark_grouping_id#15] +(21) ColumnarToRow [codegen id : 1] +Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#14, i_class#15, spark_grouping_id#16] -(21) HashAggregate [codegen id : 4] -Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#14, spark_grouping_id#15] -Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15] +(22) HashAggregate [codegen id : 1] +Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#14, i_class#15, spark_grouping_id#16] +Keys [3]: [i_category#14, i_class#15, spark_grouping_id#16] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#16, sum#17] -Results [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] +Aggregate Attributes [2]: [sum#17, sum#18] +Results [5]: [i_category#14, i_class#15, spark_grouping_id#16, sum#19, sum#20] -(22) Exchange -Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] -Arguments: hashpartitioning(i_category#13, i_class#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(23) Exchange +Input [5]: [i_category#14, i_class#15, spark_grouping_id#16, sum#19, sum#20] +Arguments: hashpartitioning(i_category#14, i_class#15, spark_grouping_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(23) HashAggregate [codegen id : 5] -Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] -Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15] +(24) HashAggregate [codegen id : 2] +Input [5]: [i_category#14, i_class#15, spark_grouping_id#16, sum#19, sum#20] +Keys [3]: [i_category#14, i_class#15, spark_grouping_id#16] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledValue(ss_ext_sales_price#3))#21] -Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS _w0#24, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS _w1#25, CASE WHEN (cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint) = 0) THEN i_category#13 END AS _w2#26] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#21, sum(UnscaledValue(ss_ext_sales_price#3))#22] +Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#21,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#22,17,2)) AS gross_margin#23, i_category#14, i_class#15, (cast((shiftright(spark_grouping_id#16, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#16, 0) & 1) as tinyint)) AS lochierarchy#24, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#21,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#22,17,2)) AS _w0#25, (cast((shiftright(spark_grouping_id#16, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#16, 0) & 1) as tinyint)) AS _w1#26, CASE WHEN (cast((shiftright(spark_grouping_id#16, 0) & 1) as tinyint) = 0) THEN i_category#14 END AS _w2#27] -(24) Exchange -Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -Arguments: hashpartitioning(_w1#25, _w2#26, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(25) Exchange +Input [7]: [gross_margin#23, i_category#14, i_class#15, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: hashpartitioning(_w1#26, _w2#27, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(25) Sort [codegen id : 6] -Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -Arguments: [_w1#25 ASC NULLS FIRST, _w2#26 ASC NULLS FIRST, _w0#24 ASC NULLS FIRST], false, 0 +(26) Sort [codegen id : 3] +Input [7]: [gross_margin#23, i_category#14, i_class#15, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 ASC NULLS FIRST], false, 0 -(26) Window -Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -Arguments: [rank(_w0#24) windowspecdefinition(_w1#25, _w2#26, _w0#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#27], [_w1#25, _w2#26], [_w0#24 ASC NULLS FIRST] +(27) Window +Input [7]: [gross_margin#23, i_category#14, i_class#15, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 ASC NULLS FIRST] -(27) Project [codegen id : 7] -Output [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] -Input [8]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26, rank_within_parent#27] +(28) Project [codegen id : 4] +Output [5]: [gross_margin#23, i_category#14, i_class#15, lochierarchy#24, rank_within_parent#28] +Input [8]: [gross_margin#23, i_category#14, i_class#15, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] -(28) TakeOrderedAndProject -Input [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] -Arguments: 100, [lochierarchy#23 DESC NULLS LAST, CASE WHEN (lochierarchy#23 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#27 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] +(29) TakeOrderedAndProject +Input [5]: [gross_margin#23, i_category#14, i_class#15, lochierarchy#24, rank_within_parent#28] +Arguments: 100, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN i_category#14 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [gross_margin#23, i_category#14, i_class#15, lochierarchy#24, rank_within_parent#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) -(29) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#28] +(30) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_year#8] 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 -(30) CometFilter -Input [2]: [d_date_sk#7, d_year#28] -Condition : ((isnotnull(d_year#28) AND (d_year#28 = 2001)) AND isnotnull(d_date_sk#7)) +(31) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(31) CometProject -Input [2]: [d_date_sk#7, d_year#28] +(32) CometProject +Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(32) ColumnarToRow [codegen id : 1] +(33) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(33) BroadcastExchange +(34) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt index 7eeb607c3b..5bdf600813 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt @@ -1,27 +1,27 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [_w1,_w2,_w0] InputAdapter Exchange [_w1,_w2] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] InputAdapter Exchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] - Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -32,20 +32,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #5 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange #6 + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt index 23c4ae742b..6eb0f14b1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt @@ -1,29 +1,30 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * ColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan parquet spark_catalog.default.catalog_sales (16) +TakeOrderedAndProject (26) ++- * HashAggregate (25) + +- Exchange (24) + +- * ColumnarToRow (23) + +- CometHashAggregate (22) + +- CometProject (21) + +- CometBroadcastHashJoin (20) + :- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- CometBroadcastExchange (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.inventory (4) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.date_dim (10) + +- CometProject (19) + +- CometFilter (18) + +- CometScan parquet spark_catalog.default.catalog_sales (17) (1) Scan parquet spark_catalog.default.item @@ -41,10 +42,7 @@ Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) A Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(4) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] - -(5) Scan parquet spark_catalog.default.inventory +(4) Scan parquet spark_catalog.default.inventory Output [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -52,128 +50,136 @@ PartitionFilters: [isnotnull(inv_date_sk#8), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Condition : (((isnotnull(inv_quantity_on_hand#7) AND (inv_quantity_on_hand#7 >= 100)) AND (inv_quantity_on_hand#7 <= 500)) AND isnotnull(inv_item_sk#6)) -(7) CometProject +(6) CometProject Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Arguments: [inv_item_sk#6, inv_date_sk#8], [inv_item_sk#6, inv_date_sk#8] -(8) ColumnarToRow [codegen id : 1] -Input [2]: [inv_item_sk#6, inv_date_sk#8] - -(9) BroadcastExchange +(7) CometBroadcastExchange Input [2]: [inv_item_sk#6, inv_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [inv_item_sk#6, inv_date_sk#8] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#6] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Right output [2]: [inv_item_sk#6, inv_date_sk#8] +Arguments: [i_item_sk#1], [inv_item_sk#6], Inner, BuildRight -(11) Project [codegen id : 3] -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] +(9) CometProject Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_item_sk#6, inv_date_sk#8] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] -(12) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#10] +(10) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_date#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] +ReadSchema: struct -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [inv_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(11) CometFilter +Input [2]: [d_date_sk#10, d_date#11] +Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-02-01)) AND (d_date#11 <= 2000-04-01)) AND isnotnull(d_date_sk#10)) -(14) Project [codegen id : 3] -Output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +(12) CometProject +Input [2]: [d_date_sk#10, d_date#11] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] + +(14) CometBroadcastHashJoin +Left output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [inv_date_sk#8], [d_date_sk#10], Inner, BuildRight + +(15) CometProject Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8, d_date_sk#10] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(15) BroadcastExchange +(16) CometBroadcastExchange Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#11, cs_sold_date_sk#12] +(17) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_item_sk#12, cs_sold_date_sk#13] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(17) CometFilter -Input [2]: [cs_item_sk#11, cs_sold_date_sk#12] -Condition : isnotnull(cs_item_sk#11) - -(18) CometProject -Input [2]: [cs_item_sk#11, cs_sold_date_sk#12] -Arguments: [cs_item_sk#11], [cs_item_sk#11] +(18) CometFilter +Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] +Condition : isnotnull(cs_item_sk#12) -(19) ColumnarToRow -Input [1]: [cs_item_sk#11] +(19) CometProject +Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] +Arguments: [cs_item_sk#12], [cs_item_sk#12] -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#11] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Right output [1]: [cs_item_sk#12] +Arguments: [i_item_sk#1], [cs_item_sk#12], Inner, BuildLeft -(21) Project [codegen id : 4] -Output [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, cs_item_sk#11] +(21) CometProject +Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, cs_item_sk#12] +Arguments: [i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_id#2, i_item_desc#3, i_current_price#4] -(22) HashAggregate [codegen id : 4] +(22) CometHashAggregate Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(23) Exchange +(23) ColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) HashAggregate [codegen id : 5] +(24) Exchange +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(25) HashAggregate [codegen id : 2] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] Aggregate Attributes: [] Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(25) TakeOrderedAndProject +(26) TakeOrderedAndProject Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (30) -+- * ColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.date_dim (26) +Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(26) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_date#13] +(27) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_date#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter -Input [2]: [d_date_sk#10, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-01)) AND (d_date#13 <= 2000-04-01)) AND isnotnull(d_date_sk#10)) +(28) CometFilter +Input [2]: [d_date_sk#10, d_date#11] +Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-02-01)) AND (d_date#11 <= 2000-04-01)) AND isnotnull(d_date_sk#10)) -(28) CometProject -Input [2]: [d_date_sk#10, d_date#13] +(29) CometProject +Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(29) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt index 65bb06348b..c3fcd79f0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt @@ -1,44 +1,38 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_current_price] InputAdapter Exchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometProject [i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,cs_item_sk] + CometBroadcastExchange #2 + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometBroadcastExchange #3 + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt index 5503439ee4..b6b11827d1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt @@ -1,51 +1,53 @@ == Physical Plan == -* HashAggregate (47) -+- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin LeftSemi BuildRight (43) - :- * BroadcastHashJoin LeftSemi BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer (7) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- BroadcastExchange (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * ColumnarToRow (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.web_sales (30) - : +- ReusedExchange (33) - +- ReusedExchange (36) +* HashAggregate (49) ++- Exchange (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftSemi BuildRight (45) + :- * BroadcastHashJoin LeftSemi BuildRight (31) + : :- * HashAggregate (17) + : : +- Exchange (16) + : : +- * ColumnarToRow (15) + : : +- CometHashAggregate (14) + : : +- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.customer (9) + : +- BroadcastExchange (30) + : +- * HashAggregate (29) + : +- Exchange (28) + : +- * ColumnarToRow (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (44) + +- * HashAggregate (43) + +- Exchange (42) + +- * ColumnarToRow (41) + +- CometHashAggregate (40) + +- CometProject (39) + +- CometBroadcastHashJoin (38) + :- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometFilter (33) + : : +- CometScan parquet spark_catalog.default.web_sales (32) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.store_sales @@ -60,262 +62,263 @@ ReadSchema: struct Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +(3) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#4, d_date#5] +(4) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#2] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] + +(6) CometBroadcastExchange +Input [2]: [d_date_sk#4, d_date#5] +Arguments: [d_date_sk#4, d_date#5] -(6) Project [codegen id : 3] -Output [2]: [ss_customer_sk#1, d_date#5] +(7) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +Right output [2]: [d_date_sk#4, d_date#5] +Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight + +(8) CometProject Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] +Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] -(7) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +(9) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Condition : isnotnull(c_customer_sk#6) - -(9) ColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +(10) CometFilter +Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Condition : isnotnull(c_customer_sk#7) -(10) BroadcastExchange -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [c_customer_sk#7, c_first_name#8, c_last_name#9] -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#6] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#1, d_date#5] +Right output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight -(12) Project [codegen id : 3] -Output [3]: [c_last_name#8, c_first_name#7, d_date#5] -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#7, c_last_name#8] +(13) CometProject +Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [c_last_name#9, c_first_name#8, d_date#5], [c_last_name#9, c_first_name#8, d_date#5] -(13) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +(14) CometHashAggregate +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] -(14) Exchange -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(15) ColumnarToRow [codegen id : 1] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] + +(16) Exchange +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +(17) HashAggregate [codegen id : 6] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] +Results [3]: [c_last_name#9, c_first_name#8, d_date#5] -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +(18) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(17) CometFilter -Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] -Condition : isnotnull(cs_bill_customer_sk#9) - -(18) ColumnarToRow [codegen id : 6] -Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +(19) CometFilter +Input [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] +Condition : isnotnull(cs_bill_customer_sk#10) -(19) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#12, d_date#13] +(20) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#13, d_date#14] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(21) CometBroadcastHashJoin +Left output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] +Right output [2]: [d_date_sk#13, d_date#14] +Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(21) Project [codegen id : 6] -Output [2]: [cs_bill_customer_sk#9, d_date#13] -Input [4]: [cs_bill_customer_sk#9, cs_sold_date_sk#10, d_date_sk#12, d_date#13] +(22) CometProject +Input [4]: [cs_bill_customer_sk#10, cs_sold_date_sk#11, d_date_sk#13, d_date#14] +Arguments: [cs_bill_customer_sk#10, d_date#14], [cs_bill_customer_sk#10, d_date#14] -(22) ReusedExchange [Reuses operator id: 10] -Output [3]: [c_customer_sk#14, c_first_name#15, c_last_name#16] +(23) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_bill_customer_sk#9] -Right keys [1]: [c_customer_sk#14] -Join type: Inner -Join condition: None +(24) CometBroadcastHashJoin +Left output [2]: [cs_bill_customer_sk#10, d_date#14] +Right output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [cs_bill_customer_sk#10], [c_customer_sk#15], Inner, BuildRight -(24) Project [codegen id : 6] -Output [3]: [c_last_name#16, c_first_name#15, d_date#13] -Input [5]: [cs_bill_customer_sk#9, d_date#13, c_customer_sk#14, c_first_name#15, c_last_name#16] +(25) CometProject +Input [5]: [cs_bill_customer_sk#10, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [c_last_name#17, c_first_name#16, d_date#14], [c_last_name#17, c_first_name#16, d_date#14] -(25) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +(26) CometHashAggregate +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(26) Exchange -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(27) ColumnarToRow [codegen id : 2] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -(27) HashAggregate [codegen id : 7] -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +(28) Exchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(29) HashAggregate [codegen id : 3] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#16, c_first_name#15, d_date#13] +Results [3]: [c_last_name#17, c_first_name#16, d_date#14] -(28) BroadcastExchange -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] +(30) BroadcastExchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] -(29) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#16, ), isnull(c_last_name#16), coalesce(c_first_name#15, ), isnull(c_first_name#15), coalesce(d_date#13, 1970-01-01), isnull(d_date#13)] +(31) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] Join type: LeftSemi Join condition: None -(30) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +(32) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#18), dynamicpruningexpression(ws_sold_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] -Condition : isnotnull(ws_bill_customer_sk#17) - -(32) ColumnarToRow [codegen id : 10] -Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +(33) CometFilter +Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Condition : isnotnull(ws_bill_customer_sk#18) -(33) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#20, d_date#21] +(34) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#21, d_date#22] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(35) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Right output [2]: [d_date_sk#21, d_date#22] +Arguments: [ws_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight -(35) Project [codegen id : 10] -Output [2]: [ws_bill_customer_sk#17, d_date#21] -Input [4]: [ws_bill_customer_sk#17, ws_sold_date_sk#18, d_date_sk#20, d_date#21] +(36) CometProject +Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#21, d_date#22] +Arguments: [ws_bill_customer_sk#18, d_date#22], [ws_bill_customer_sk#18, d_date#22] -(36) ReusedExchange [Reuses operator id: 10] -Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] +(37) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_bill_customer_sk#17] -Right keys [1]: [c_customer_sk#22] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, d_date#22] +Right output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [ws_bill_customer_sk#18], [c_customer_sk#23], Inner, BuildRight -(38) Project [codegen id : 10] -Output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Input [5]: [ws_bill_customer_sk#17, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] +(39) CometProject +Input [5]: [ws_bill_customer_sk#18, d_date#22, c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [c_last_name#25, c_first_name#24, d_date#22], [c_last_name#25, c_first_name#24, d_date#22] -(39) HashAggregate [codegen id : 10] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +(40) CometHashAggregate +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(40) Exchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(41) ColumnarToRow [codegen id : 4] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] + +(42) Exchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: hashpartitioning(c_last_name#25, c_first_name#24, d_date#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(41) HashAggregate [codegen id : 11] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +(43) HashAggregate [codegen id : 5] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] +Results [3]: [c_last_name#25, c_first_name#24, d_date#22] -(42) BroadcastExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] +(44) BroadcastExchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] +(45) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#25, ), isnull(c_last_name#25), coalesce(c_first_name#24, ), isnull(c_first_name#24), coalesce(d_date#22, 1970-01-01), isnull(d_date#22)] Join type: LeftSemi Join condition: None -(44) Project [codegen id : 12] +(46) Project [codegen id : 6] Output: [] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(45) HashAggregate [codegen id : 12] +(47) HashAggregate [codegen id : 6] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#25] -Results [1]: [count#26] +Aggregate Attributes [1]: [count#26] +Results [1]: [count#27] -(46) Exchange -Input [1]: [count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(48) Exchange +Input [1]: [count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(47) HashAggregate [codegen id : 13] -Input [1]: [count#26] +(49) HashAggregate [codegen id : 7] +Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#27] -Results [1]: [count(1)#27 AS count(1)#28] +Aggregate Attributes [1]: [count(1)#28] +Results [1]: [count(1)#28 AS count(1)#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(48) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +(50) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) +(51) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) -(50) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +(52) CometProject +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(52) BroadcastExchange +(54) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 +Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt index 315afe6602..cfac83844a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (13) +WholeStageCodegen (7) HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 - WholeStageCodegen (12) + WholeStageCodegen (6) HashAggregate [count,count] Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] @@ -10,14 +10,14 @@ WholeStageCodegen (13) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometProject [ss_customer_sk,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -28,54 +28,48 @@ WholeStageCodegen (13) CometProject [d_date_sk,d_date] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange #4 + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange #5 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (3) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,d_date] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) + BroadcastExchange #8 + WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [c_last_name,c_first_name,d_date] #9 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometProject [ws_bill_customer_sk,d_date] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt index d851f61bf1..5d6b73e889 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt @@ -1,48 +1,52 @@ == Physical Plan == -* Sort (44) -+- Exchange (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (23) - : +- * Filter (22) - : +- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (41) - +- * Project (40) - +- * Filter (39) - +- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * ColumnarToRow (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (27) - : +- ReusedExchange (30) - +- ReusedExchange (33) +* Sort (48) ++- Exchange (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (24) + : +- * Filter (23) + : +- * HashAggregate (22) + : +- Exchange (21) + : +- * HashAggregate (20) + : +- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.item (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.warehouse (8) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.date_dim (13) + +- BroadcastExchange (45) + +- * Project (44) + +- * Filter (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * ColumnarToRow (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometFilter (26) + : : : +- CometScan parquet spark_catalog.default.inventory (25) + : : +- ReusedExchange (27) + : +- ReusedExchange (30) + +- CometBroadcastExchange (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (1) Scan parquet spark_catalog.default.inventory @@ -57,262 +61,282 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [1]: [i_item_sk#6] Condition : isnotnull(i_item_sk#6) -(6) ColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#6] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [1]: [i_item_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#6] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [1]: [i_item_sk#6] +Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight -(9) Project [codegen id : 4] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +(7) CometProject Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -(10) Scan parquet spark_catalog.default.warehouse +(8) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Condition : isnotnull(w_warehouse_sk#7) -(12) ColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [w_warehouse_sk#7, w_warehouse_name#8] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#7] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight -(15) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +(12) CometProject Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -(16) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#9, d_moy#10] +(13) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(14) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] +(15) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +(16) CometBroadcastExchange +Input [2]: [d_date_sk#9, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11] + +(17) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +Right output [2]: [d_date_sk#9, d_moy#11] +Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight + +(18) CometProject +Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] +Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] + +(19) ColumnarToRow [codegen id : 1] +Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] + +(20) HashAggregate [codegen id : 1] +Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] +Aggregate Attributes [5]: [n#12, avg#13, m2#14, sum#15, count#16] +Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#17, avg#18, m2#19, sum#20, count#21] -(20) Exchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) Exchange +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#17, avg#18, m2#19, sum#20, count#21] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +(22) HashAggregate [codegen id : 4] +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#17, avg#18, m2#19, sum#20, count#21] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] -Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#22, avg(inv_quantity_on_hand#3)#23] +Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stddev_samp(cast(inv_quantity_on_hand#3 as double))#22 AS stdev#24, avg(inv_quantity_on_hand#3)#23 AS mean#25] -(22) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END +(23) Filter [codegen id : 4] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#24, mean#25] +Condition : CASE WHEN (mean#25 = 0.0) THEN false ELSE ((stdev#24 / mean#25) > 1.0) END -(23) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] +(24) Project [codegen id : 4] +Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, CASE WHEN (mean#25 = 0.0) THEN null ELSE (stdev#24 / mean#25) END AS cov#26] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#24, mean#25] -(24) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] +(25) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(inv_date_sk#30), dynamicpruningexpression(inv_date_sk#30 IN dynamicpruning#31)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) +(26) CometFilter +Input [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] +Condition : (isnotnull(inv_item_sk#27) AND isnotnull(inv_warehouse_sk#28)) -(26) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] - -(27) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#31] - -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#26] -Right keys [1]: [i_item_sk#31] -Join type: Inner -Join condition: None +(27) ReusedExchange [Reuses operator id: 5] +Output [1]: [i_item_sk#32] -(29) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] +(28) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] +Right output [1]: [i_item_sk#32] +Arguments: [inv_item_sk#27], [i_item_sk#32], Inner, BuildRight -(30) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(29) CometProject +Input [5]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32] +Arguments: [inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32], [inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32] -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#27] -Right keys [1]: [w_warehouse_sk#32] -Join type: Inner -Join condition: None +(30) ReusedExchange [Reuses operator id: 10] +Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] -(32) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] +(31) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32] +Right output [2]: [w_warehouse_sk#33, w_warehouse_name#34] +Arguments: [inv_warehouse_sk#28], [w_warehouse_sk#33], Inner, BuildRight -(33) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#34, d_moy#35] +(32) CometProject +Input [6]: [inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Arguments: [inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34], [inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(33) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#35, d_year#36, d_moy#37] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] +ReadSchema: struct -(35) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] - -(36) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] -Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] -Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(37) Exchange -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(38) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] - -(39) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] -Condition : CASE WHEN (mean#47 = 0.0) THEN false ELSE ((stdev#46 / mean#47) > 1.0) END - -(40) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, CASE WHEN (mean#47 = 0.0) THEN null ELSE (stdev#46 / mean#47) END AS cov#48] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] - -(41) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(42) BroadcastHashJoin [codegen id : 10] +(34) CometFilter +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) + +(35) CometProject +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Arguments: [d_date_sk#35, d_moy#37], [d_date_sk#35, d_moy#37] + +(36) CometBroadcastExchange +Input [2]: [d_date_sk#35, d_moy#37] +Arguments: [d_date_sk#35, d_moy#37] + +(37) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Right output [2]: [d_date_sk#35, d_moy#37] +Arguments: [inv_date_sk#30], [d_date_sk#35], Inner, BuildRight + +(38) CometProject +Input [7]: [inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] +Arguments: [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37], [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] + +(39) ColumnarToRow [codegen id : 2] +Input [5]: [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] + +(40) HashAggregate [codegen id : 2] +Input [5]: [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] +Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#29 as double)), partial_avg(inv_quantity_on_hand#29)] +Aggregate Attributes [5]: [n#38, avg#39, m2#40, sum#41, count#42] +Results [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#43, avg#44, m2#45, sum#46, count#47] + +(41) Exchange +Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#43, avg#44, m2#45, sum#46, count#47] +Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(42) HashAggregate [codegen id : 3] +Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#43, avg#44, m2#45, sum#46, count#47] +Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#29 as double)), avg(inv_quantity_on_hand#29)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#29 as double))#22, avg(inv_quantity_on_hand#29)#23] +Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_quantity_on_hand#29 as double))#22 AS stdev#48, avg(inv_quantity_on_hand#29)#23 AS mean#49] + +(43) Filter [codegen id : 3] +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#48, mean#49] +Condition : CASE WHEN (mean#49 = 0.0) THEN false ELSE ((stdev#48 / mean#49) > 1.0) END + +(44) Project [codegen id : 3] +Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, CASE WHEN (mean#49 = 0.0) THEN null ELSE (stdev#48 / mean#49) END AS cov#50] +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#48, mean#49] + +(45) BroadcastExchange +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=3] + +(46) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] +Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] Join type: Inner Join condition: None -(43) Exchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(47) Exchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, cov#26, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#25 ASC NULLS FIRST, cov#26 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, mean#49 ASC NULLS FIRST, cov#50 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(44) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST], true, 0 +(48) Sort [codegen id : 5] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, cov#26, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50] +Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#25 ASC NULLS FIRST, cov#26 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, mean#49 ASC NULLS FIRST, cov#50 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(45) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#49, d_moy#10] +(49) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [3]: [d_date_sk#9, d_year#49, d_moy#10] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#10)) AND (d_year#49 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) +(50) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) -(47) CometProject -Input [3]: [d_date_sk#9, d_year#49, d_moy#10] -Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] +(51) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(48) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#10] +(52) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_moy#11] -(49) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(53) BroadcastExchange +Input [2]: [d_date_sk#9, d_moy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#30 IN dynamicpruning#31 +BroadcastExchange (58) ++- * ColumnarToRow (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) -(50) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#50, d_moy#35] +(54) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#35, d_year#36, d_moy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter -Input [3]: [d_date_sk#34, d_year#50, d_moy#35] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#35)) AND (d_year#50 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) +(55) CometFilter +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) -(52) CometProject -Input [3]: [d_date_sk#34, d_year#50, d_moy#35] -Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] +(56) CometProject +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Arguments: [d_date_sk#35, d_moy#37], [d_date_sk#35, d_moy#37] -(53) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_moy#35] +(57) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#35, d_moy#37] -(54) BroadcastExchange -Input [2]: [d_date_sk#34, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +(58) BroadcastExchange +Input [2]: [d_date_sk#35, d_moy#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt index 002266e76e..ff321c523f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt @@ -1,24 +1,24 @@ -WholeStageCodegen (11) +WholeStageCodegen (5) Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] InputAdapter Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) + WholeStageCodegen (4) BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,53 +29,47 @@ WholeStageCodegen (11) CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 + CometBroadcastExchange #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #6 + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) + BroadcastExchange #7 + WholeStageCodegen (3) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 + WholeStageCodegen (2) HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange #10 + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt index a4184150e6..38f56d6044 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt @@ -1,48 +1,52 @@ == Physical Plan == -* Sort (44) -+- Exchange (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (23) - : +- * Filter (22) - : +- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (41) - +- * Project (40) - +- * Filter (39) - +- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * ColumnarToRow (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (27) - : +- ReusedExchange (30) - +- ReusedExchange (33) +* Sort (48) ++- Exchange (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (24) + : +- * Filter (23) + : +- * HashAggregate (22) + : +- Exchange (21) + : +- * HashAggregate (20) + : +- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.item (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.warehouse (8) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.date_dim (13) + +- BroadcastExchange (45) + +- * Project (44) + +- * Filter (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * ColumnarToRow (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometFilter (26) + : : : +- CometScan parquet spark_catalog.default.inventory (25) + : : +- ReusedExchange (27) + : +- ReusedExchange (30) + +- CometBroadcastExchange (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (1) Scan parquet spark_catalog.default.inventory @@ -57,262 +61,282 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [1]: [i_item_sk#6] Condition : isnotnull(i_item_sk#6) -(6) ColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#6] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [1]: [i_item_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#6] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [1]: [i_item_sk#6] +Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight -(9) Project [codegen id : 4] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +(7) CometProject Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -(10) Scan parquet spark_catalog.default.warehouse +(8) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Condition : isnotnull(w_warehouse_sk#7) -(12) ColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [w_warehouse_sk#7, w_warehouse_name#8] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#7] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight -(15) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +(12) CometProject Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -(16) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#9, d_moy#10] +(13) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(14) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] +(15) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +(16) CometBroadcastExchange +Input [2]: [d_date_sk#9, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11] + +(17) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +Right output [2]: [d_date_sk#9, d_moy#11] +Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight + +(18) CometProject +Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] +Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] + +(19) ColumnarToRow [codegen id : 1] +Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] + +(20) HashAggregate [codegen id : 1] +Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] +Aggregate Attributes [5]: [n#12, avg#13, m2#14, sum#15, count#16] +Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#17, avg#18, m2#19, sum#20, count#21] -(20) Exchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) Exchange +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#17, avg#18, m2#19, sum#20, count#21] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +(22) HashAggregate [codegen id : 4] +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#17, avg#18, m2#19, sum#20, count#21] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] -Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#22, avg(inv_quantity_on_hand#3)#23] +Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stddev_samp(cast(inv_quantity_on_hand#3 as double))#22 AS stdev#24, avg(inv_quantity_on_hand#3)#23 AS mean#25] -(22) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : (CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END AND CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.5) END) +(23) Filter [codegen id : 4] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#24, mean#25] +Condition : (CASE WHEN (mean#25 = 0.0) THEN false ELSE ((stdev#24 / mean#25) > 1.0) END AND CASE WHEN (mean#25 = 0.0) THEN false ELSE ((stdev#24 / mean#25) > 1.5) END) -(23) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] +(24) Project [codegen id : 4] +Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, CASE WHEN (mean#25 = 0.0) THEN null ELSE (stdev#24 / mean#25) END AS cov#26] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#24, mean#25] -(24) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] +(25) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(inv_date_sk#30), dynamicpruningexpression(inv_date_sk#30 IN dynamicpruning#31)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) +(26) CometFilter +Input [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] +Condition : (isnotnull(inv_item_sk#27) AND isnotnull(inv_warehouse_sk#28)) -(26) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] - -(27) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#31] - -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#26] -Right keys [1]: [i_item_sk#31] -Join type: Inner -Join condition: None +(27) ReusedExchange [Reuses operator id: 5] +Output [1]: [i_item_sk#32] -(29) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] +(28) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30] +Right output [1]: [i_item_sk#32] +Arguments: [inv_item_sk#27], [i_item_sk#32], Inner, BuildRight -(30) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(29) CometProject +Input [5]: [inv_item_sk#27, inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32] +Arguments: [inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32], [inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32] -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#27] -Right keys [1]: [w_warehouse_sk#32] -Join type: Inner -Join condition: None +(30) ReusedExchange [Reuses operator id: 10] +Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] -(32) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] +(31) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32] +Right output [2]: [w_warehouse_sk#33, w_warehouse_name#34] +Arguments: [inv_warehouse_sk#28], [w_warehouse_sk#33], Inner, BuildRight -(33) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#34, d_moy#35] +(32) CometProject +Input [6]: [inv_warehouse_sk#28, inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Arguments: [inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34], [inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(33) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#35, d_year#36, d_moy#37] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] +ReadSchema: struct -(35) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] - -(36) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] -Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] -Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(37) Exchange -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(38) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] - -(39) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] -Condition : CASE WHEN (mean#47 = 0.0) THEN false ELSE ((stdev#46 / mean#47) > 1.0) END - -(40) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, CASE WHEN (mean#47 = 0.0) THEN null ELSE (stdev#46 / mean#47) END AS cov#48] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] - -(41) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(42) BroadcastHashJoin [codegen id : 10] +(34) CometFilter +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) + +(35) CometProject +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Arguments: [d_date_sk#35, d_moy#37], [d_date_sk#35, d_moy#37] + +(36) CometBroadcastExchange +Input [2]: [d_date_sk#35, d_moy#37] +Arguments: [d_date_sk#35, d_moy#37] + +(37) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Right output [2]: [d_date_sk#35, d_moy#37] +Arguments: [inv_date_sk#30], [d_date_sk#35], Inner, BuildRight + +(38) CometProject +Input [7]: [inv_quantity_on_hand#29, inv_date_sk#30, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] +Arguments: [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37], [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] + +(39) ColumnarToRow [codegen id : 2] +Input [5]: [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] + +(40) HashAggregate [codegen id : 2] +Input [5]: [inv_quantity_on_hand#29, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] +Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#29 as double)), partial_avg(inv_quantity_on_hand#29)] +Aggregate Attributes [5]: [n#38, avg#39, m2#40, sum#41, count#42] +Results [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#43, avg#44, m2#45, sum#46, count#47] + +(41) Exchange +Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#43, avg#44, m2#45, sum#46, count#47] +Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(42) HashAggregate [codegen id : 3] +Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#43, avg#44, m2#45, sum#46, count#47] +Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#29 as double)), avg(inv_quantity_on_hand#29)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#29 as double))#22, avg(inv_quantity_on_hand#29)#23] +Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_quantity_on_hand#29 as double))#22 AS stdev#48, avg(inv_quantity_on_hand#29)#23 AS mean#49] + +(43) Filter [codegen id : 3] +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#48, mean#49] +Condition : CASE WHEN (mean#49 = 0.0) THEN false ELSE ((stdev#48 / mean#49) > 1.0) END + +(44) Project [codegen id : 3] +Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, CASE WHEN (mean#49 = 0.0) THEN null ELSE (stdev#48 / mean#49) END AS cov#50] +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#48, mean#49] + +(45) BroadcastExchange +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=3] + +(46) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] +Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] Join type: Inner Join condition: None -(43) Exchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(47) Exchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, cov#26, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#25 ASC NULLS FIRST, cov#26 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, mean#49 ASC NULLS FIRST, cov#50 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(44) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST], true, 0 +(48) Sort [codegen id : 5] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, cov#26, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, cov#50] +Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#25 ASC NULLS FIRST, cov#26 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, mean#49 ASC NULLS FIRST, cov#50 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(45) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#49, d_moy#10] +(49) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [3]: [d_date_sk#9, d_year#49, d_moy#10] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#10)) AND (d_year#49 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) +(50) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) -(47) CometProject -Input [3]: [d_date_sk#9, d_year#49, d_moy#10] -Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] +(51) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(48) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#10] +(52) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_moy#11] -(49) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(53) BroadcastExchange +Input [2]: [d_date_sk#9, d_moy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#30 IN dynamicpruning#31 +BroadcastExchange (58) ++- * ColumnarToRow (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) -(50) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#50, d_moy#35] +(54) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#35, d_year#36, d_moy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter -Input [3]: [d_date_sk#34, d_year#50, d_moy#35] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#35)) AND (d_year#50 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) +(55) CometFilter +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) -(52) CometProject -Input [3]: [d_date_sk#34, d_year#50, d_moy#35] -Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] +(56) CometProject +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Arguments: [d_date_sk#35, d_moy#37], [d_date_sk#35, d_moy#37] -(53) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_moy#35] +(57) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#35, d_moy#37] -(54) BroadcastExchange -Input [2]: [d_date_sk#34, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +(58) BroadcastExchange +Input [2]: [d_date_sk#35, d_moy#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt index 002266e76e..ff321c523f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt @@ -1,24 +1,24 @@ -WholeStageCodegen (11) +WholeStageCodegen (5) Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] InputAdapter Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) + WholeStageCodegen (4) BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,53 +29,47 @@ WholeStageCodegen (11) CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 + CometBroadcastExchange #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #6 + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) + BroadcastExchange #7 + WholeStageCodegen (3) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 + WholeStageCodegen (2) HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange #10 + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt index a854a1041b..d25660dba0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt @@ -1,112 +1,110 @@ == Physical Plan == -TakeOrderedAndProject (108) -+- * Project (107) - +- * BroadcastHashJoin Inner BuildRight (106) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) +TakeOrderedAndProject (106) ++- * Project (105) + +- * BroadcastHashJoin Inner BuildRight (104) + :- * Project (88) + : +- * BroadcastHashJoin Inner BuildRight (87) : :- * Project (70) : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * Project (52) - : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : :- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Filter (16) - : : : : : +- * HashAggregate (15) - : : : : : +- Exchange (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (10) - : : : : +- BroadcastExchange (32) - : : : : +- * HashAggregate (31) - : : : : +- Exchange (30) - : : : : +- * HashAggregate (29) - : : : : +- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * Project (25) - : : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : : :- * ColumnarToRow (19) - : : : : : : +- CometFilter (18) - : : : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : : : +- BroadcastExchange (23) - : : : : : +- * ColumnarToRow (22) - : : : : : +- CometFilter (21) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : : : +- ReusedExchange (26) - : : : +- BroadcastExchange (50) - : : : +- * Filter (49) - : : : +- * HashAggregate (48) - : : : +- Exchange (47) - : : : +- * HashAggregate (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * ColumnarToRow (36) - : : : : : +- CometFilter (35) - : : : : : +- CometScan parquet spark_catalog.default.customer (34) - : : : : +- BroadcastExchange (40) - : : : : +- * ColumnarToRow (39) - : : : : +- CometFilter (38) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (37) - : : : +- ReusedExchange (43) + : : :- * Project (53) + : : : +- * BroadcastHashJoin Inner BuildRight (52) + : : : :- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Filter (17) + : : : : : +- * HashAggregate (16) + : : : : : +- Exchange (15) + : : : : : +- * HashAggregate (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : : : +- BroadcastExchange (34) + : : : : +- * HashAggregate (33) + : : : : +- Exchange (32) + : : : : +- * HashAggregate (31) + : : : : +- * ColumnarToRow (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (19) + : : : : : : +- CometScan parquet spark_catalog.default.customer (18) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometFilter (21) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : : : +- CometBroadcastExchange (27) + : : : : +- CometFilter (26) + : : : : +- CometScan parquet spark_catalog.default.date_dim (25) + : : : +- BroadcastExchange (51) + : : : +- * Filter (50) + : : : +- * HashAggregate (49) + : : : +- Exchange (48) + : : : +- * HashAggregate (47) + : : : +- * ColumnarToRow (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometProject (42) + : : : : +- CometBroadcastHashJoin (41) + : : : : :- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.customer (36) + : : : : +- CometBroadcastExchange (40) + : : : : +- CometFilter (39) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (38) + : : : +- ReusedExchange (43) : : +- BroadcastExchange (68) : : +- * HashAggregate (67) : : +- Exchange (66) : : +- * HashAggregate (65) - : : +- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * ColumnarToRow (55) - : : : : +- CometFilter (54) - : : : : +- CometScan parquet spark_catalog.default.customer (53) - : : : +- BroadcastExchange (59) - : : : +- * ColumnarToRow (58) - : : : +- CometFilter (57) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (56) - : : +- ReusedExchange (62) - : +- BroadcastExchange (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (79) - : : +- * BroadcastHashJoin Inner BuildRight (78) - : : :- * ColumnarToRow (73) - : : : +- CometFilter (72) - : : : +- CometScan parquet spark_catalog.default.customer (71) - : : +- BroadcastExchange (77) - : : +- * ColumnarToRow (76) - : : +- CometFilter (75) - : : +- CometScan parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (80) - +- BroadcastExchange (105) - +- * HashAggregate (104) - +- Exchange (103) - +- * HashAggregate (102) - +- * Project (101) - +- * BroadcastHashJoin Inner BuildRight (100) - :- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * ColumnarToRow (92) - : : +- CometFilter (91) - : : +- CometScan parquet spark_catalog.default.customer (90) - : +- BroadcastExchange (96) - : +- * ColumnarToRow (95) - : +- CometFilter (94) - : +- CometScan parquet spark_catalog.default.web_sales (93) - +- ReusedExchange (99) + : : +- * ColumnarToRow (64) + : : +- CometProject (63) + : : +- CometBroadcastHashJoin (62) + : : :- CometProject (60) + : : : +- CometBroadcastHashJoin (59) + : : : :- CometFilter (55) + : : : : +- CometScan parquet spark_catalog.default.customer (54) + : : : +- CometBroadcastExchange (58) + : : : +- CometFilter (57) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (56) + : : +- ReusedExchange (61) + : +- BroadcastExchange (86) + : +- * Filter (85) + : +- * HashAggregate (84) + : +- Exchange (83) + : +- * HashAggregate (82) + : +- * ColumnarToRow (81) + : +- CometProject (80) + : +- CometBroadcastHashJoin (79) + : :- CometProject (77) + : : +- CometBroadcastHashJoin (76) + : : :- CometFilter (72) + : : : +- CometScan parquet spark_catalog.default.customer (71) + : : +- CometBroadcastExchange (75) + : : +- CometFilter (74) + : : +- CometScan parquet spark_catalog.default.web_sales (73) + : +- ReusedExchange (78) + +- BroadcastExchange (103) + +- * HashAggregate (102) + +- Exchange (101) + +- * HashAggregate (100) + +- * ColumnarToRow (99) + +- CometProject (98) + +- CometBroadcastHashJoin (97) + :- CometProject (95) + : +- CometBroadcastHashJoin (94) + : :- CometFilter (90) + : : +- CometScan parquet spark_catalog.default.customer (89) + : +- CometBroadcastExchange (93) + : +- CometFilter (92) + : +- CometScan parquet spark_catalog.default.web_sales (91) + +- ReusedExchange (96) (1) Scan parquet spark_catalog.default.customer @@ -120,10 +118,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] Condition : isnotnull(ss_customer_sk#9) -(6) ColumnarToRow [codegen id : 1] -Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Right output [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Arguments: [c_customer_sk#1], [ss_customer_sk#9], Inner, BuildRight -(9) Project [codegen id : 3] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +(7) CometProject Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -(10) ReusedExchange [Reuses operator id: 112] +(8) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_year#17] +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 -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#14] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) + +(10) CometBroadcastExchange +Input [2]: [d_date_sk#16, d_year#17] +Arguments: [d_date_sk#16, d_year#17] -(12) Project [codegen id : 3] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] +(11) CometBroadcastHashJoin +Left output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Right output [2]: [d_date_sk#16, d_year#17] +Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight + +(12) CometProject Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14, d_date_sk#16, d_year#17] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] + +(13) ColumnarToRow [codegen id : 1] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] -(13) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] Functions [1]: [partial_sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] -(14) Exchange +(15) Exchange Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 24] +(16) HashAggregate [codegen id : 12] Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] Functions [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))] Aggregate Attributes [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22] Results [2]: [c_customer_id#2 AS customer_id#23, sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22 AS year_total#24] -(16) Filter [codegen id : 24] +(17) Filter [codegen id : 12] Input [2]: [customer_id#23, year_total#24] Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.000000)) -(17) Scan parquet spark_catalog.default.customer +(18) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter +(19) CometFilter Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_customer_id#26)) -(19) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] - (20) Scan parquet spark_catalog.default.store_sales Output [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] Batched: true @@ -213,79 +215,86 @@ ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None +(26) CometFilter +Input [2]: [d_date_sk#40, d_year#41] +Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) + +(27) CometBroadcastExchange +Input [2]: [d_date_sk#40, d_year#41] +Arguments: [d_date_sk#40, d_year#41] -(28) Project [codegen id : 6] -Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] +(28) CometBroadcastHashJoin +Left output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Right output [2]: [d_date_sk#40, d_year#41] +Arguments: [ss_sold_date_sk#38], [d_date_sk#40], Inner, BuildRight + +(29) CometProject Input [14]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] +Arguments: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41], [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] + +(30) ColumnarToRow [codegen id : 2] +Input [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] -(29) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] Functions [1]: [partial_sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] Aggregate Attributes [2]: [sum#42, isEmpty#43] Results [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] -(30) Exchange +(32) Exchange Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] -Arguments: hashpartitioning(c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(31) HashAggregate [codegen id : 7] +(33) HashAggregate [codegen id : 3] Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] Functions [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] Aggregate Attributes [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22] Results [8]: [c_customer_id#26 AS customer_id#46, c_first_name#27 AS customer_first_name#47, c_last_name#28 AS customer_last_name#48, c_preferred_cust_flag#29 AS customer_preferred_cust_flag#49, c_birth_country#30 AS customer_birth_country#50, c_login#31 AS customer_login#51, c_email_address#32 AS customer_email_address#52, sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22 AS year_total#53] -(32) BroadcastExchange +(34) BroadcastExchange Input [8]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 24] +(35) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#46] Join type: Inner Join condition: None -(34) Scan parquet spark_catalog.default.customer +(36) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter +(37) CometFilter Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] Condition : (isnotnull(c_customer_sk#54) AND isnotnull(c_customer_id#55)) -(36) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] - -(37) Scan parquet spark_catalog.default.catalog_sales +(38) Scan parquet spark_catalog.default.catalog_sales Output [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] Batched: true Location: InMemoryFileIndex [] @@ -293,90 +302,85 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#67), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(38) CometFilter +(39) CometFilter Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] Condition : isnotnull(cs_bill_customer_sk#62) -(39) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] - -(40) BroadcastExchange +(40) CometBroadcastExchange Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#54] -Right keys [1]: [cs_bill_customer_sk#62] -Join type: Inner -Join condition: None +(41) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] +Right output [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Arguments: [c_customer_sk#54], [cs_bill_customer_sk#62], Inner, BuildRight -(42) Project [codegen id : 10] -Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +(42) CometProject Input [14]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67], [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -(43) ReusedExchange [Reuses operator id: 112] +(43) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#69, d_year#70] -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#67] -Right keys [1]: [d_date_sk#69] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Right output [2]: [d_date_sk#69, d_year#70] +Arguments: [cs_sold_date_sk#67], [d_date_sk#69], Inner, BuildRight -(45) Project [codegen id : 10] -Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] +(45) CometProject Input [14]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67, d_date_sk#69, d_year#70] +Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70], [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] -(46) HashAggregate [codegen id : 10] +(46) ColumnarToRow [codegen id : 4] +Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] + +(47) HashAggregate [codegen id : 4] Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] Functions [1]: [partial_sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] Aggregate Attributes [2]: [sum#71, isEmpty#72] Results [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] -(47) Exchange +(48) Exchange Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(48) HashAggregate [codegen id : 11] +(49) HashAggregate [codegen id : 5] Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] Functions [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] Aggregate Attributes [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75] Results [2]: [c_customer_id#55 AS customer_id#76, sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75 AS year_total#77] -(49) Filter [codegen id : 11] +(50) Filter [codegen id : 5] Input [2]: [customer_id#76, year_total#77] Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.000000)) -(50) BroadcastExchange +(51) BroadcastExchange Input [2]: [customer_id#76, year_total#77] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(51) BroadcastHashJoin [codegen id : 24] +(52) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#76] Join type: Inner Join condition: None -(52) Project [codegen id : 24] +(53) Project [codegen id : 12] Output [11]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77] Input [12]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, customer_id#76, year_total#77] -(53) Scan parquet spark_catalog.default.customer +(54) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter +(55) CometFilter Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] Condition : (isnotnull(c_customer_sk#78) AND isnotnull(c_customer_id#79)) -(55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] - (56) Scan parquet spark_catalog.default.catalog_sales Output [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] Batched: true @@ -389,37 +393,35 @@ ReadSchema: struct 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END) -(70) Project [codegen id : 24] +(70) Project [codegen id : 12] Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100] Input [13]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77, customer_id#99, year_total#100] @@ -462,10 +464,7 @@ ReadSchema: struct -(75) CometFilter +(74) CometFilter Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] Condition : isnotnull(ws_bill_customer_sk#109) -(76) ColumnarToRow [codegen id : 16] +(75) CometBroadcastExchange Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +Arguments: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] -(77) BroadcastExchange -Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#101] -Right keys [1]: [ws_bill_customer_sk#109] -Join type: Inner -Join condition: None +(76) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Right output [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +Arguments: [c_customer_sk#101], [ws_bill_customer_sk#109], Inner, BuildRight -(79) Project [codegen id : 18] -Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +(77) CometProject Input [14]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +Arguments: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114], [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] -(80) ReusedExchange [Reuses operator id: 112] +(78) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#116, d_year#117] -(81) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#114] -Right keys [1]: [d_date_sk#116] -Join type: Inner -Join condition: None +(79) CometBroadcastHashJoin +Left output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +Right output [2]: [d_date_sk#116, d_year#117] +Arguments: [ws_sold_date_sk#114], [d_date_sk#116], Inner, BuildRight -(82) Project [codegen id : 18] -Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] +(80) CometProject Input [14]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114, d_date_sk#116, d_year#117] +Arguments: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117], [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] + +(81) ColumnarToRow [codegen id : 8] +Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] -(83) HashAggregate [codegen id : 18] +(82) HashAggregate [codegen id : 8] Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] Functions [1]: [partial_sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] Aggregate Attributes [2]: [sum#118, isEmpty#119] Results [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] -(84) Exchange +(83) Exchange Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] -Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(85) HashAggregate [codegen id : 19] +(84) HashAggregate [codegen id : 9] Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] Functions [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] Aggregate Attributes [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122] Results [2]: [c_customer_id#102 AS customer_id#123, sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122 AS year_total#124] -(86) Filter [codegen id : 19] +(85) Filter [codegen id : 9] Input [2]: [customer_id#123, year_total#124] Condition : (isnotnull(year_total#124) AND (year_total#124 > 0.000000)) -(87) BroadcastExchange +(86) BroadcastExchange Input [2]: [customer_id#123, year_total#124] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=9] -(88) BroadcastHashJoin [codegen id : 24] +(87) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#123] Join type: Inner Join condition: None -(89) Project [codegen id : 24] +(88) Project [codegen id : 12] Output [11]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124] Input [12]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, customer_id#123, year_total#124] -(90) Scan parquet spark_catalog.default.customer +(89) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(91) CometFilter +(90) CometFilter Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] Condition : (isnotnull(c_customer_sk#125) AND isnotnull(c_customer_id#126)) -(92) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] - -(93) Scan parquet spark_catalog.default.web_sales +(91) Scan parquet spark_catalog.default.web_sales Output [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] Batched: true Location: InMemoryFileIndex [] @@ -565,134 +559,132 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#138), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(94) CometFilter +(92) CometFilter Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] Condition : isnotnull(ws_bill_customer_sk#133) -(95) ColumnarToRow [codegen id : 20] +(93) CometBroadcastExchange Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +Arguments: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] -(96) BroadcastExchange -Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] +(94) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +Right output [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +Arguments: [c_customer_sk#125], [ws_bill_customer_sk#133], Inner, BuildRight -(97) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#125] -Right keys [1]: [ws_bill_customer_sk#133] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 22] -Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +(95) CometProject Input [14]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +Arguments: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138], [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] -(99) ReusedExchange [Reuses operator id: 116] +(96) ReusedExchange [Reuses operator id: 27] Output [2]: [d_date_sk#140, d_year#141] -(100) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#138] -Right keys [1]: [d_date_sk#140] -Join type: Inner -Join condition: None +(97) CometBroadcastHashJoin +Left output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +Right output [2]: [d_date_sk#140, d_year#141] +Arguments: [ws_sold_date_sk#138], [d_date_sk#140], Inner, BuildRight -(101) Project [codegen id : 22] -Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] +(98) CometProject Input [14]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138, d_date_sk#140, d_year#141] +Arguments: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141], [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] + +(99) ColumnarToRow [codegen id : 10] +Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] -(102) HashAggregate [codegen id : 22] +(100) HashAggregate [codegen id : 10] Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] Functions [1]: [partial_sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] Aggregate Attributes [2]: [sum#142, isEmpty#143] Results [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] -(103) Exchange +(101) Exchange Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] -Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(104) HashAggregate [codegen id : 23] +(102) HashAggregate [codegen id : 11] Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] Functions [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] Aggregate Attributes [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122] Results [2]: [c_customer_id#126 AS customer_id#146, sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122 AS year_total#147] -(105) BroadcastExchange +(103) BroadcastExchange Input [2]: [customer_id#146, year_total#147] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(106) BroadcastHashJoin [codegen id : 24] +(104) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#146] Join type: Inner Join condition: (CASE WHEN (year_total#77 > 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#124 > 0.000000) THEN (year_total#147 / year_total#124) END) -(107) Project [codegen id : 24] +(105) Project [codegen id : 12] Output [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] Input [13]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124, customer_id#146, year_total#147] -(108) TakeOrderedAndProject +(106) TakeOrderedAndProject Input [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] Arguments: 100, [customer_id#46 ASC NULLS FIRST, customer_first_name#47 ASC NULLS FIRST, customer_last_name#48 ASC NULLS FIRST, customer_preferred_cust_flag#49 ASC NULLS FIRST, customer_birth_country#50 ASC NULLS FIRST, customer_login#51 ASC NULLS FIRST, customer_email_address#52 ASC NULLS FIRST], [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 -BroadcastExchange (112) -+- * ColumnarToRow (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 +BroadcastExchange (110) ++- * ColumnarToRow (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(109) Scan parquet spark_catalog.default.date_dim +(107) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_year#17] 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 -(110) CometFilter +(108) CometFilter Input [2]: [d_date_sk#16, d_year#17] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(111) ColumnarToRow [codegen id : 1] +(109) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] -(112) BroadcastExchange +(110) BroadcastExchange Input [2]: [d_date_sk#16, d_year#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 -BroadcastExchange (116) -+- * ColumnarToRow (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (114) ++- * ColumnarToRow (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) -(113) Scan parquet spark_catalog.default.date_dim +(111) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#40, d_year#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter +(112) CometFilter Input [2]: [d_date_sk#40, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) -(115) ColumnarToRow [codegen id : 1] +(113) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#40, d_year#41] -(116) BroadcastExchange +(114) BroadcastExchange Input [2]: [d_date_sk#40, d_year#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 38 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#15 Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#39 -Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#114 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#114 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#39 +Subquery:6 Hosting operator id = 91 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#39 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt index 99e255a0e4..0918da5757 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - WholeStageCodegen (24) + WholeStageCodegen (12) Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] @@ -13,167 +13,141 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 + WholeStageCodegen (2) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange #7 CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #12 + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #13 + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #15 + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 InputAdapter - BroadcastExchange #14 - WholeStageCodegen (19) + BroadcastExchange #16 + WholeStageCodegen (9) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 - WholeStageCodegen (18) + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 + WholeStageCodegen (8) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #18 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #17 - WholeStageCodegen (23) + BroadcastExchange #19 + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 - WholeStageCodegen (22) + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #21 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt index d51d63d8a5..a43b8716f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt @@ -1,25 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (19) ++- * HashAggregate (18) + +- Exchange (17) + +- * HashAggregate (16) + +- * ColumnarToRow (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- CometBroadcastExchange (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.store_sales (4) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.item (9) (1) Scan parquet spark_catalog.default.date_dim @@ -37,10 +35,7 @@ Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) Input [3]: [d_date_sk#1, d_year#2, d_moy#3] Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] -(4) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#1, d_year#2] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -48,78 +43,73 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Condition : isnotnull(ss_item_sk#4) -(7) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [d_date_sk#1, d_year#2] +Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight -(10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +(8) CometProject Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -(11) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] 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 -(12) CometFilter +(10) CometFilter Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) -(13) CometProject +(11) CometProject Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] Arguments: [i_item_sk#7, i_category_id#8, i_category#9], [i_item_sk#7, i_category_id#8, i_category#9] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [3]: [i_item_sk#7, i_category_id#8, i_category#9] +Arguments: [i_item_sk#7, i_category_id#8, i_category#9] -(15) BroadcastExchange -Input [3]: [i_item_sk#7, i_category_id#8, i_category#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(13) CometBroadcastHashJoin +Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +Right output [3]: [i_item_sk#7, i_category_id#8, i_category#9] +Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#4] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] +(14) CometProject Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#9] +Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] + +(15) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] -(18) HashAggregate [codegen id : 3] +(16) HashAggregate [codegen id : 1] Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] Keys [3]: [d_year#2, i_category_id#8, i_category#9] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] -(19) Exchange +(17) Exchange Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] Keys [3]: [d_year#2, i_category_id#8, i_category#9] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] Results [4]: [d_year#2, i_category_id#8, i_category#9, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum(ss_ext_sales_price)#14] -(21) TakeOrderedAndProject +(19) TakeOrderedAndProject Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] Arguments: 100, [sum(ss_ext_sales_price)#14 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt index 67906b8c7a..e3967a972e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt @@ -1,31 +1,23 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] InputAdapter Exchange [d_year,i_category_id,i_category] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] - Project [d_year,ss_ext_sales_price,i_category_id,i_category] - 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] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_year] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_category_id,i_category] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt index e892aa4694..abe134d264 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt @@ -1,25 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.store (11) +TakeOrderedAndProject (19) ++- * HashAggregate (18) + +- Exchange (17) + +- * HashAggregate (16) + +- * ColumnarToRow (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- CometBroadcastExchange (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.store_sales (4) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.store (9) (1) Scan parquet spark_catalog.default.date_dim @@ -37,10 +35,7 @@ Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] Arguments: [d_date_sk#1, d_day_name#3], [d_date_sk#1, d_day_name#3] -(4) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#1, d_day_name#3] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -48,78 +43,73 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Condition : isnotnull(ss_store_sk#4) -(7) ColumnarToRow [codegen id : 1] -Input [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [d_date_sk#1, d_day_name#3] +Right output [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight -(10) Project [codegen id : 3] -Output [3]: [d_day_name#3, ss_store_sk#4, ss_sales_price#5] +(8) CometProject Input [5]: [d_date_sk#1, d_day_name#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_day_name#3, ss_store_sk#4, ss_sales_price#5], [d_day_name#3, ss_store_sk#4, ss_sales_price#5] -(11) Scan parquet spark_catalog.default.store +(9) Scan parquet spark_catalog.default.store Output [4]: [s_store_sk#7, s_store_id#8, s_store_name#9, s_gmt_offset#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [4]: [s_store_sk#7, s_store_id#8, s_store_name#9, s_gmt_offset#10] Condition : ((isnotnull(s_gmt_offset#10) AND (s_gmt_offset#10 = -5.00)) AND isnotnull(s_store_sk#7)) -(13) CometProject +(11) CometProject Input [4]: [s_store_sk#7, s_store_id#8, s_store_name#9, s_gmt_offset#10] Arguments: [s_store_sk#7, s_store_id#8, s_store_name#9], [s_store_sk#7, s_store_id#8, s_store_name#9] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [3]: [s_store_sk#7, s_store_id#8, s_store_name#9] +Arguments: [s_store_sk#7, s_store_id#8, s_store_name#9] -(15) BroadcastExchange -Input [3]: [s_store_sk#7, s_store_id#8, s_store_name#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(13) CometBroadcastHashJoin +Left output [3]: [d_day_name#3, ss_store_sk#4, ss_sales_price#5] +Right output [3]: [s_store_sk#7, s_store_id#8, s_store_name#9] +Arguments: [ss_store_sk#4], [s_store_sk#7], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 3] -Output [4]: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] +(14) CometProject Input [6]: [d_day_name#3, ss_store_sk#4, ss_sales_price#5, s_store_sk#7, s_store_id#8, s_store_name#9] +Arguments: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9], [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] + +(15) ColumnarToRow [codegen id : 1] +Input [4]: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] -(18) HashAggregate [codegen id : 3] +(16) HashAggregate [codegen id : 1] Input [4]: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] Keys [2]: [s_store_name#9, s_store_id#8] Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))] Aggregate Attributes [7]: [sum#11, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17] Results [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] -(19) Exchange +(17) Exchange Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(s_store_name#9, s_store_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(s_store_name#9, s_store_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [2]: [s_store_name#9, s_store_id#8] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))] Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31] Results [9]: [s_store_name#9, s_store_id#8, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25,17,2) AS sun_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26,17,2) AS mon_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27,17,2) AS tue_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28,17,2) AS wed_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29,17,2) AS thu_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30,17,2) AS fri_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31,17,2) AS sat_sales#38] -(21) TakeOrderedAndProject +(19) TakeOrderedAndProject Input [9]: [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] Arguments: 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#32 ASC NULLS FIRST, mon_sales#33 ASC NULLS FIRST, tue_sales#34 ASC NULLS FIRST, wed_sales#35 ASC NULLS FIRST, thu_sales#36 ASC NULLS FIRST, fri_sales#37 ASC NULLS FIRST, sat_sales#38 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt index ef20430969..63945612a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt @@ -1,31 +1,23 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [s_store_name,s_store_id] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,ss_sales_price,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [d_day_name,ss_store_sk,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [d_day_name,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + CometBroadcastExchange #2 + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [s_store_sk,s_store_id,s_store_name] + CometFilter [s_gmt_offset,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt index d0d74569be..f128499e34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt @@ -6,28 +6,28 @@ TakeOrderedAndProject (36) +- * Project (32) +- * Filter (31) +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.customer (4) - : : : +- BroadcastExchange (13) - : : : +- * ColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (16) - : +- BroadcastExchange (22) - : +- * ColumnarToRow (21) + :- * ColumnarToRow (24) + : +- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.customer (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.customer_address (8) + : : +- CometBroadcastExchange (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.date_dim (13) + : +- CometBroadcastExchange (21) : +- CometFilter (20) : +- CometScan parquet spark_catalog.default.item (19) +- BroadcastExchange (29) @@ -49,163 +49,166 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [c_customer_sk#7, c_current_addr_sk#8] Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [2]: [c_customer_sk#7, c_current_addr_sk#8] +Arguments: [c_customer_sk#7, c_current_addr_sk#8] -(7) BroadcastExchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] +Right output [2]: [c_customer_sk#7, c_current_addr_sk#8] +Arguments: [ws_bill_customer_sk#3], [c_customer_sk#7], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_bill_customer_sk#3] -Right keys [1]: [c_customer_sk#7] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 6] -Output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] +(7) CometProject Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] +Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] -(10) Scan parquet spark_catalog.default.customer_address +(8) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] Condition : isnotnull(ca_address_sk#9) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] +Arguments: [ca_address_sk#9, ca_city#10, ca_zip#11] -(13) BroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] +Right output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] +Arguments: [c_current_addr_sk#8], [ca_address_sk#9], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_addr_sk#8] -Right keys [1]: [ca_address_sk#9] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 6] -Output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11] +(12) CometProject Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#11] +Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11] -(16) ReusedExchange [Reuses operator id: 41] -Output [1]: [d_date_sk#12] +(13) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#5] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(14) CometFilter +Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] +Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) -(18) Project [codegen id : 6] -Output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11] +(15) CometProject +Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] +Arguments: [d_date_sk#12], [d_date_sk#12] + +(16) CometBroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: [d_date_sk#12] + +(17) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11] +Right output [1]: [d_date_sk#12] +Arguments: [ws_sold_date_sk#5], [d_date_sk#12], Inner, BuildRight + +(18) CometProject Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11, d_date_sk#12] +Arguments: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11], [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11] (19) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#15, i_item_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (20) CometFilter -Input [2]: [i_item_sk#13, i_item_id#14] -Condition : isnotnull(i_item_sk#13) +Input [2]: [i_item_sk#15, i_item_id#16] +Condition : isnotnull(i_item_sk#15) -(21) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] +(21) CometBroadcastExchange +Input [2]: [i_item_sk#15, i_item_id#16] +Arguments: [i_item_sk#15, i_item_id#16] -(22) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(22) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11] +Right output [2]: [i_item_sk#15, i_item_id#16] +Arguments: [ws_item_sk#2], [i_item_sk#15], Inner, BuildRight -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(23) CometProject +Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11, i_item_sk#15, i_item_id#16] +Arguments: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16], [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] -(24) Project [codegen id : 6] -Output [4]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#14] -Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11, i_item_sk#13, i_item_id#14] +(24) ColumnarToRow [codegen id : 2] +Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] (25) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] +Output [2]: [i_item_sk#17, i_item_id#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] ReadSchema: struct (26) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : i_item_sk#15 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#17, i_item_id#18] +Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) (27) CometProject -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: [i_item_id#16], [i_item_id#16] +Input [2]: [i_item_sk#17, i_item_id#18] +Arguments: [i_item_id#18], [i_item_id#18] -(28) ColumnarToRow [codegen id : 5] -Input [1]: [i_item_id#16] +(28) ColumnarToRow [codegen id : 1] +Input [1]: [i_item_id#18] (29) BroadcastExchange -Input [1]: [i_item_id#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] +Input [1]: [i_item_id#18] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#16] +(30) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [i_item_id#16] +Right keys [1]: [i_item_id#18] Join type: ExistenceJoin(exists#1) Join condition: None -(31) Filter [codegen id : 6] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#14, exists#1] +(31) Filter [codegen id : 2] +Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16, exists#1] Condition : (substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#11] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#14, exists#1] +Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16, exists#1] -(33) HashAggregate [codegen id : 6] +(33) HashAggregate [codegen id : 2] Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#11] Keys [2]: [ca_zip#11, ca_city#10] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#17] -Results [3]: [ca_zip#11, ca_city#10, sum#18] +Aggregate Attributes [1]: [sum#19] +Results [3]: [ca_zip#11, ca_city#10, sum#20] (34) Exchange -Input [3]: [ca_zip#11, ca_city#10, sum#18] -Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [3]: [ca_zip#11, ca_city#10, sum#20] +Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(35) HashAggregate [codegen id : 7] -Input [3]: [ca_zip#11, ca_city#10, sum#18] +(35) HashAggregate [codegen id : 3] +Input [3]: [ca_zip#11, ca_city#10, sum#20] Keys [2]: [ca_zip#11, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#19] -Results [3]: [ca_zip#11, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#19,17,2) AS sum(ws_sales_price)#20] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#21] +Results [3]: [ca_zip#11, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#21,17,2) AS sum(ws_sales_price)#22] (36) TakeOrderedAndProject -Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] -Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] +Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#22] +Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#11, ca_city#10, sum(ws_sales_price)#22] ===== Subqueries ===== @@ -218,18 +221,18 @@ BroadcastExchange (41) (37) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#21, d_qoy#22] +Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (38) CometFilter -Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] -Condition : ((((isnotnull(d_qoy#22) AND isnotnull(d_year#21)) AND (d_qoy#22 = 2)) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#12)) +Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] +Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) (39) CometProject -Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] +Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] Arguments: [d_date_sk#12], [d_date_sk#12] (40) ColumnarToRow [codegen id : 1] @@ -237,6 +240,6 @@ Input [1]: [d_date_sk#12] (41) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt index 383cbb7e3b..c376c4fcd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt @@ -1,23 +1,23 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] InputAdapter Exchange [ca_zip,ca_city] #1 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] BroadcastHashJoin [i_item_id,i_item_id] - Project [ws_sales_price,ca_city,ca_zip,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] CometFilter [ws_bill_customer_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -28,32 +28,22 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometProject [d_date_sk] CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #3 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #4 + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt index e07e2ab242..71f97587a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt @@ -1,43 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) +TakeOrderedAndProject (42) ++- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) :- * Project (35) : +- * BroadcastHashJoin Inner BuildRight (34) : :- * HashAggregate (29) : : +- Exchange (28) : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) + : : +- * ColumnarToRow (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometProject (20) + : : : +- CometBroadcastHashJoin (19) + : : : :- CometProject (14) + : : : : +- CometBroadcastHashJoin (13) + : : : : :- CometProject (8) + : : : : : +- CometBroadcastHashJoin (7) + : : : : : :- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- CometBroadcastExchange (6) + : : : : : +- CometProject (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : : +- CometBroadcastExchange (12) + : : : : +- CometProject (11) + : : : : +- CometFilter (10) + : : : : +- CometScan parquet spark_catalog.default.store (9) + : : : +- CometBroadcastExchange (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.household_demographics (15) + : : +- CometBroadcastExchange (23) : : +- CometFilter (22) : : +- CometScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (33) : +- * ColumnarToRow (32) : +- CometFilter (31) : +- CometScan parquet spark_catalog.default.customer (30) - +- ReusedExchange (36) + +- BroadcastExchange (39) + +- * ColumnarToRow (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.customer_address (36) (1) Scan parquet spark_catalog.default.store_sales @@ -52,207 +55,225 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#10] +(4) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] + +(7) CometBroadcastHashJoin +Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight -(6) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +(8) CometProject Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_city#12] +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#13, s_city#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#11, s_city#12] -Condition : (s_city#12 IN (Fairview,Midway) AND isnotnull(s_store_sk#11)) +(10) CometFilter +Input [2]: [s_store_sk#13, s_city#14] +Condition : (s_city#14 IN (Fairview,Midway) AND isnotnull(s_store_sk#13)) -(9) CometProject -Input [2]: [s_store_sk#11, s_city#12] -Arguments: [s_store_sk#11], [s_store_sk#11] +(11) CometProject +Input [2]: [s_store_sk#13, s_city#14] +Arguments: [s_store_sk#13], [s_store_sk#13] -(10) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#11] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#13] +Arguments: [s_store_sk#13] -(11) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +Right output [1]: [s_store_sk#13] +Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight -(13) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11] +(14) CometProject +Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -(14) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] +(15) Scan parquet spark_catalog.default.household_demographics +Output [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : (((hd_dep_count#14 = 4) OR (hd_vehicle_count#15 = 3)) AND isnotnull(hd_demo_sk#13)) +(16) CometFilter +Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] +Condition : (((hd_dep_count#16 = 4) OR (hd_vehicle_count#17 = 3)) AND isnotnull(hd_demo_sk#15)) -(16) CometProject -Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#13], [hd_demo_sk#13] +(17) CometProject +Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] +Arguments: [hd_demo_sk#15], [hd_demo_sk#15] -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#13] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#15] +Arguments: [hd_demo_sk#15] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(19) CometBroadcastHashJoin +Left output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +Right output [1]: [hd_demo_sk#15] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#15], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#13] +(20) CometProject +Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#15] +Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] (21) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_city#17] +Output [2]: [ca_address_sk#18, ca_city#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] ReadSchema: struct (22) CometFilter -Input [2]: [ca_address_sk#16, ca_city#17] -Condition : (isnotnull(ca_address_sk#16) AND isnotnull(ca_city#17)) +Input [2]: [ca_address_sk#18, ca_city#19] +Condition : (isnotnull(ca_address_sk#18) AND isnotnull(ca_city#19)) -(23) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#16, ca_city#17] +(23) CometBroadcastExchange +Input [2]: [ca_address_sk#18, ca_city#19] +Arguments: [ca_address_sk#18, ca_city#19] -(24) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_city#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(24) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +Right output [2]: [ca_address_sk#18, ca_city#19] +Arguments: [ss_addr_sk#3], [ca_address_sk#18], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None +(25) CometProject +Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#18, ca_city#19] +Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] -(26) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#16, ca_city#17] +(26) ColumnarToRow [codegen id : 1] +Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] -(27) HashAggregate [codegen id : 5] -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] +(27) HashAggregate [codegen id : 1] +Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum#18, sum#19] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] +Aggregate Attributes [2]: [sum#20, sum#21] +Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#22, sum#23] (28) Exchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#22, sum#23] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 8] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] +(29) HashAggregate [codegen id : 4] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#22, sum#23] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#22, sum(UnscaledValue(ss_net_profit#7))#23] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#17 AS bought_city#24, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#22,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#23,17,2) AS profit#26] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#24, sum(UnscaledValue(ss_net_profit#7))#25] +Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#19 AS bought_city#26, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#24,17,2) AS amt#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#25,17,2) AS profit#28] (30) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +Output [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (31) CometFilter -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) +Input [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#30)) -(32) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +(32) ColumnarToRow [codegen id : 2] +Input [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] (33) BroadcastExchange -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Input [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(34) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#27] +Right keys [1]: [c_customer_sk#29] Join type: Inner Join condition: None -(35) Project [codegen id : 8] -Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +(35) Project [codegen id : 4] +Output [7]: [ss_ticket_number#5, bought_city#26, amt#27, profit#28, c_current_addr_sk#30, c_first_name#31, c_last_name#32] +Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#26, amt#27, profit#28, c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] + +(36) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#33, ca_city#34] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] +ReadSchema: struct -(36) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#31, ca_city#32] +(37) CometFilter +Input [2]: [ca_address_sk#33, ca_city#34] +Condition : (isnotnull(ca_address_sk#33) AND isnotnull(ca_city#34)) + +(38) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#33, ca_city#34] + +(39) BroadcastExchange +Input [2]: [ca_address_sk#33, ca_city#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#28] -Right keys [1]: [ca_address_sk#31] +(40) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [c_current_addr_sk#30] +Right keys [1]: [ca_address_sk#33] Join type: Inner -Join condition: NOT (ca_city#32 = bought_city#24) +Join condition: NOT (ca_city#34 = bought_city#26) -(38) Project [codegen id : 8] -Output [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_city#32] +(41) Project [codegen id : 4] +Output [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#26, ss_ticket_number#5, amt#27, profit#28] +Input [9]: [ss_ticket_number#5, bought_city#26, amt#27, profit#28, c_current_addr_sk#30, c_first_name#31, c_last_name#32, ca_address_sk#33, ca_city#34] -(39) TakeOrderedAndProject -Input [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Arguments: 100, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#24 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] +(42) TakeOrderedAndProject +Input [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#26, ss_ticket_number#5, amt#27, profit#28] +Arguments: 100, [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, ca_city#34 ASC NULLS FIRST, bought_city#26 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#32, c_first_name#31, ca_city#34, bought_city#26, ss_ticket_number#5, amt#27, profit#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(40) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#33, d_dow#34] +(43) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_dow#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [3]: [d_date_sk#10, d_year#33, d_dow#34] -Condition : ((d_dow#34 IN (6,0) AND d_year#33 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) +(44) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(42) CometProject -Input [3]: [d_date_sk#10, d_year#33, d_dow#34] +(45) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(43) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(44) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt index 04c59a2d35..718557c106 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - WholeStageCodegen (8) + WholeStageCodegen (4) Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] @@ -7,18 +7,18 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] InputAdapter Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,37 +29,32 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu CometProject [d_date_sk] CometFilter [d_dow,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_dow,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange #4 + CometProject [s_store_sk] + CometFilter [s_city,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometBroadcastExchange #5 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #6 + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 + BroadcastExchange #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt index fca7362ce8..12f171f319 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt @@ -14,22 +14,22 @@ TakeOrderedAndProject (45) : : +- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) + : : +- * ColumnarToRow (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.store (13) : +- BroadcastExchange (35) @@ -57,10 +57,7 @@ ReadSchema: struct Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] - -(4) Scan parquet spark_catalog.default.store_sales +(3) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -68,39 +65,46 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Condition : (isnotnull(ss_item_sk#4) AND isnotnull(ss_store_sk#5)) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#4] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Right output [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: [i_item_sk#1], [ss_item_sk#4], Inner, BuildRight -(9) Project [codegen id : 4] -Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +(7) CometProject Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7], [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_year#10, d_moy#11] + +(11) CometBroadcastHashJoin +Left output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(12) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] +(12) CometProject Input [8]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] +Arguments: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11], [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] (13) Scan parquet spark_catalog.default.store Output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] @@ -113,24 +117,23 @@ ReadSchema: struct Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] Condition : ((isnotnull(s_store_sk#12) AND isnotnull(s_store_name#13)) AND isnotnull(s_company_name#14)) -(15) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [s_store_sk#12, s_store_name#13, s_company_name#14] -(16) BroadcastExchange -Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] +Right output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [ss_store_sk#5], [s_store_sk#12], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] +(17) CometProject Input [9]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14], [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] + +(18) ColumnarToRow [codegen id : 1] +Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 1] Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#6))] @@ -139,9 +142,9 @@ Results [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_yea (20) Exchange Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 2] Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] @@ -150,9 +153,9 @@ Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_yea (22) Exchange Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(23) Sort [codegen id : 6] +(23) Sort [codegen id : 3] Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 @@ -160,7 +163,7 @@ Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_nam Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] +(25) Filter [codegen id : 4] Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) @@ -168,18 +171,18 @@ Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] -(27) Filter [codegen id : 22] +(27) Filter [codegen id : 13] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(28) Project [codegen id : 22] +(28) Project [codegen id : 13] Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] (29) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(30) HashAggregate [codegen id : 12] +(30) HashAggregate [codegen id : 6] Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] @@ -188,9 +191,9 @@ Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_y (31) Exchange Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 13] +(32) Sort [codegen id : 7] Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 @@ -198,28 +201,28 @@ Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_n Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(34) Project [codegen id : 14] +(34) Project [codegen id : 8] Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] (35) BroadcastExchange Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 22] +(36) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] +(37) Project [codegen id : 13] Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] (38) ReusedExchange [Reuses operator id: 31] Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -(39) Sort [codegen id : 20] +(39) Sort [codegen id : 11] Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 @@ -227,21 +230,21 @@ Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_n Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(41) Project [codegen id : 21] +(41) Project [codegen id : 12] Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] (42) BroadcastExchange Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 22] +(43) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] +(44) Project [codegen id : 13] Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] @@ -251,7 +254,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_ ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (49) +- * ColumnarToRow (48) +- CometFilter (47) @@ -274,6 +277,6 @@ Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt index 80b8da7b14..66eb5a4cba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,74 +8,67 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) + BroadcastExchange #7 + WholeStageCodegen (8) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) + WholeStageCodegen (7) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) + Exchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (6) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] InputAdapter ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #9 + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) + WholeStageCodegen (11) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt index 718f3fb317..5197d99f8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt @@ -2,31 +2,31 @@ * HashAggregate (28) +- Exchange (27) +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.customer_demographics (10) - : +- BroadcastExchange (20) - : +- * ColumnarToRow (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.customer_address (16) - +- ReusedExchange (23) + +- * ColumnarToRow (25) + +- CometProject (24) + +- CometBroadcastHashJoin (23) + :- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.store (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.customer_demographics (8) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.customer_address (13) + +- CometBroadcastExchange (22) + +- CometProject (21) + +- CometFilter (20) + +- CometScan parquet spark_catalog.default.date_dim (19) (1) Scan parquet spark_catalog.default.store_sales @@ -41,127 +41,130 @@ ReadSchema: struct= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) -(3) ColumnarToRow [codegen id : 5] -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(4) Scan parquet spark_catalog.default.store +(3) Scan parquet spark_catalog.default.store Output [1]: [s_store_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [1]: [s_store_sk#9] Condition : isnotnull(s_store_sk#9) -(6) ColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#9] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [1]: [s_store_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [s_store_sk#9] -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Right output [1]: [s_store_sk#9] +Arguments: [ss_store_sk#3], [s_store_sk#9], Inner, BuildRight -(9) Project [codegen id : 5] -Output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +(7) CometProject Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] +Arguments: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -(10) Scan parquet spark_catalog.default.customer_demographics +(8) Scan parquet spark_catalog.default.customer_demographics Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree )),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree ))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College )))] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] Condition : (isnotnull(cd_demo_sk#10) AND ((((cd_marital_status#11 = M) AND (cd_education_status#12 = 4 yr Degree )) OR ((cd_marital_status#11 = D) AND (cd_education_status#12 = 2 yr Degree ))) OR ((cd_marital_status#11 = S) AND (cd_education_status#12 = College )))) -(12) ColumnarToRow [codegen id : 2] -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: ((((((cd_marital_status#11 = M) AND (cd_education_status#12 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#11 = D) AND (cd_education_status#12 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#11 = S) AND (cd_education_status#12 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) +(11) CometBroadcastHashJoin +Left output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Right output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] +Arguments: [ss_cdemo_sk#1], [cd_demo_sk#10], Inner, ((((((cd_marital_status#11 = M) AND (cd_education_status#12 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#11 = D) AND (cd_education_status#12 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#11 = S) AND (cd_education_status#12 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))), BuildRight -(15) Project [codegen id : 5] -Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] +(12) CometProject Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] +Arguments: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7], [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] -(16) Scan parquet spark_catalog.default.customer_address +(13) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] 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, [CO,OH,TX]),In(ca_state, [KY,MN,OR])),In(ca_state, [CA,MS,VA]))] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((ca_state#14 IN (CO,OH,TX) OR ca_state#14 IN (OR,MN,KY)) OR ca_state#14 IN (VA,CA,MS))) -(18) CometProject +(15) CometProject Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] Arguments: [ca_address_sk#13, ca_state#14], [ca_address_sk#13, ca_state#14] -(19) ColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#13, ca_state#14] - -(20) BroadcastExchange +(16) CometBroadcastExchange Input [2]: [ca_address_sk#13, ca_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [ca_address_sk#13, ca_state#14] -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#13] -Join type: Inner -Join condition: ((((ca_state#14 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#14 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#14 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))) +(17) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] +Right output [2]: [ca_address_sk#13, ca_state#14] +Arguments: [ss_addr_sk#2], [ca_address_sk#13], Inner, ((((ca_state#14 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#14 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#14 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))), BuildRight -(22) Project [codegen id : 5] -Output [2]: [ss_quantity#4, ss_sold_date_sk#7] +(18) CometProject Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#13, ca_state#14] +Arguments: [ss_quantity#4, ss_sold_date_sk#7], [ss_quantity#4, ss_sold_date_sk#7] -(23) ReusedExchange [Reuses operator id: 33] -Output [1]: [d_date_sk#16] +(19) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_year#17] +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 + +(20) CometFilter +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) + +(21) CometProject +Input [2]: [d_date_sk#16, d_year#17] +Arguments: [d_date_sk#16], [d_date_sk#16] + +(22) CometBroadcastExchange +Input [1]: [d_date_sk#16] +Arguments: [d_date_sk#16] -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [2]: [ss_quantity#4, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#16] +Arguments: [ss_sold_date_sk#7], [d_date_sk#16], Inner, BuildRight -(25) Project [codegen id : 5] -Output [1]: [ss_quantity#4] +(24) CometProject Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#16] +Arguments: [ss_quantity#4], [ss_quantity#4] + +(25) ColumnarToRow [codegen id : 1] +Input [1]: [ss_quantity#4] -(26) HashAggregate [codegen id : 5] +(26) HashAggregate [codegen id : 1] Input [1]: [ss_quantity#4] Keys: [] Functions [1]: [partial_sum(ss_quantity#4)] -Aggregate Attributes [1]: [sum#17] -Results [1]: [sum#18] +Aggregate Attributes [1]: [sum#18] +Results [1]: [sum#19] (27) Exchange -Input [1]: [sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Input [1]: [sum#19] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 6] -Input [1]: [sum#18] +(28) HashAggregate [codegen id : 2] +Input [1]: [sum#19] Keys: [] Functions [1]: [sum(ss_quantity#4)] -Aggregate Attributes [1]: [sum(ss_quantity#4)#19] -Results [1]: [sum(ss_quantity#4)#19 AS sum(ss_quantity)#20] +Aggregate Attributes [1]: [sum(ss_quantity#4)#20] +Results [1]: [sum(ss_quantity#4)#20 AS sum(ss_quantity)#21] ===== Subqueries ===== @@ -174,18 +177,18 @@ BroadcastExchange (33) (29) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_year#21] +Output [2]: [d_date_sk#16, d_year#17] 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 (30) CometFilter -Input [2]: [d_date_sk#16, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) (31) CometProject -Input [2]: [d_date_sk#16, d_year#21] +Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] (32) ColumnarToRow [codegen id : 1] @@ -193,6 +196,6 @@ Input [1]: [d_date_sk#16] (33) BroadcastExchange Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt index 4022da74f1..30f9822ab3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt @@ -1,19 +1,19 @@ -WholeStageCodegen (6) +WholeStageCodegen (2) HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] InputAdapter Exchange #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_quantity] [sum,sum] - Project [ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_quantity,ss_sold_date_sk] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -24,27 +24,17 @@ WholeStageCodegen (6) CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #4 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastExchange #5 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt index bbb550e057..124a22ef96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt @@ -1,78 +1,81 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- Union (70) - :- * Project (23) - : +- * Filter (22) - : +- Window (21) - : +- * Sort (20) - : +- Window (19) - : +- * Sort (18) - : +- Exchange (17) - : +- * HashAggregate (16) - : +- Exchange (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.web_returns (5) - : +- ReusedExchange (11) - :- * Project (46) - : +- * Filter (45) - : +- Window (44) - : +- * Sort (43) - : +- Window (42) - : +- * Sort (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometScan parquet spark_catalog.default.catalog_returns (28) - : +- ReusedExchange (34) - +- * Project (69) - +- * Filter (68) - +- Window (67) - +- * Sort (66) - +- Window (65) - +- * Sort (64) - +- Exchange (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * ColumnarToRow (56) - : +- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometBroadcastExchange (50) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan parquet spark_catalog.default.store_sales (47) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.store_returns (51) - +- ReusedExchange (57) +TakeOrderedAndProject (77) ++- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * Sort (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- Exchange (18) + : +- * HashAggregate (17) + : +- * ColumnarToRow (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * Sort (44) + : +- Exchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * ColumnarToRow (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * Sort (67) + +- Exchange (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * ColumnarToRow (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) Scan parquet spark_catalog.default.web_sales @@ -113,344 +116,357 @@ Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_am (8) CometBroadcastHashJoin Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner +Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft (9) CometProject Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(10) ColumnarToRow [codegen id : 2] -Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +(10) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] +ReadSchema: struct + +(11) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(11) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#13] +(12) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Arguments: [d_date_sk#13], [d_date_sk#13] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: [d_date_sk#13] -(12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight -(13) Project [codegen id : 2] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] +(15) CometProject Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] + +(16) ColumnarToRow [codegen id : 1] +Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(14) HashAggregate [codegen id : 2] +(17) HashAggregate [codegen id : 1] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Aggregate Attributes [6]: [sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] +Results [7]: [ws_item_sk#1, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -(15) Exchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(18) Exchange +Input [7]: [ws_item_sk#1, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(16) HashAggregate [codegen id : 3] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(19) HashAggregate [codegen id : 2] +Input [7]: [ws_item_sk#1, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] +Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#28, sum(coalesce(ws_quantity#3, 0))#29, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#30, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#31] +Results [3]: [ws_item_sk#1 AS item#32, (cast(sum(coalesce(wr_return_quantity#10, 0))#28 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#29 as decimal(15,4))) AS return_ratio#33, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#30 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#31 as decimal(15,4))) AS currency_ratio#34] -(17) Exchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] +(20) Exchange +Input [3]: [item#32, return_ratio#33, currency_ratio#34] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(18) Sort [codegen id : 4] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 +(21) Sort [codegen id : 3] +Input [3]: [item#32, return_ratio#33, currency_ratio#34] +Arguments: [return_ratio#33 ASC NULLS FIRST], false, 0 -(19) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] +(22) Window +Input [3]: [item#32, return_ratio#33, currency_ratio#34] +Arguments: [rank(return_ratio#33) windowspecdefinition(return_ratio#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#35], [return_ratio#33 ASC NULLS FIRST] -(20) Sort [codegen id : 5] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 4] +Input [4]: [item#32, return_ratio#33, currency_ratio#34, return_rank#35] +Arguments: [currency_ratio#34 ASC NULLS FIRST], false, 0 -(21) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] +(24) Window +Input [4]: [item#32, return_ratio#33, currency_ratio#34, return_rank#35] +Arguments: [rank(currency_ratio#34) windowspecdefinition(currency_ratio#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#36], [currency_ratio#34 ASC NULLS FIRST] -(22) Filter [codegen id : 6] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) +(25) Filter [codegen id : 5] +Input [5]: [item#32, return_ratio#33, currency_ratio#34, return_rank#35, currency_rank#36] +Condition : ((return_rank#35 <= 10) OR (currency_rank#36 <= 10)) -(23) Project [codegen id : 6] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +(26) Project [codegen id : 5] +Output [5]: [web AS channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Input [5]: [item#32, return_ratio#33, currency_ratio#34, return_rank#35, currency_rank#36] -(24) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +(27) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_net_profit#42, cs_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(25) CometFilter -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) +(28) CometFilter +Input [6]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_net_profit#42, cs_sold_date_sk#43] +Condition : (((((((isnotnull(cs_net_profit#42) AND isnotnull(cs_net_paid#41)) AND isnotnull(cs_quantity#40)) AND (cs_net_profit#42 > 1.00)) AND (cs_net_paid#41 > 0.00)) AND (cs_quantity#40 > 0)) AND isnotnull(cs_order_number#39)) AND isnotnull(cs_item_sk#38)) -(26) CometProject -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(29) CometProject +Input [6]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_net_profit#42, cs_sold_date_sk#43] +Arguments: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43], [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43] -(27) CometBroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(30) CometBroadcastExchange +Input [5]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43] +Arguments: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43] -(28) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +(31) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48, cr_returned_date_sk#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(29) CometFilter -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) - -(30) CometProject -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(31) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner - -(32) CometProject -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] - -(33) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] - -(34) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#48] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] - -(37) HashAggregate [codegen id : 8] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] -Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] - -(38) Exchange -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(39) HashAggregate [codegen id : 9] -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] -Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] - -(40) Exchange -Input [3]: [item#65, return_ratio#66, currency_ratio#67] +(32) CometFilter +Input [5]: [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48, cr_returned_date_sk#49] +Condition : (((isnotnull(cr_return_amount#48) AND (cr_return_amount#48 > 10000.00)) AND isnotnull(cr_order_number#46)) AND isnotnull(cr_item_sk#45)) + +(33) CometProject +Input [5]: [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48, cr_returned_date_sk#49] +Arguments: [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48], [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48] + +(34) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43] +Right output [4]: [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48] +Arguments: [cs_order_number#39, cs_item_sk#38], [cr_order_number#46, cr_item_sk#45], Inner, BuildLeft + +(35) CometProject +Input [9]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48] +Arguments: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_return_quantity#47, cr_return_amount#48], [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_return_quantity#47, cr_return_amount#48] + +(36) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#50] + +(37) CometBroadcastHashJoin +Left output [6]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_return_quantity#47, cr_return_amount#48] +Right output [1]: [d_date_sk#50] +Arguments: [cs_sold_date_sk#43], [d_date_sk#50], Inner, BuildRight + +(38) CometProject +Input [7]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_return_quantity#47, cr_return_amount#48, d_date_sk#50] +Arguments: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48], [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48] + +(39) ColumnarToRow [codegen id : 6] +Input [5]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48] + +(40) HashAggregate [codegen id : 6] +Input [5]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48] +Keys [1]: [cs_item_sk#38] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#47, 0)), partial_sum(coalesce(cs_quantity#40, 0)), partial_sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#41 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] +Results [7]: [cs_item_sk#38, sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] + +(41) Exchange +Input [7]: [cs_item_sk#38, sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Arguments: hashpartitioning(cs_item_sk#38, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(42) HashAggregate [codegen id : 7] +Input [7]: [cs_item_sk#38, sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Keys [1]: [cs_item_sk#38] +Functions [4]: [sum(coalesce(cr_return_quantity#47, 0)), sum(coalesce(cs_quantity#40, 0)), sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#41 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#47, 0))#63, sum(coalesce(cs_quantity#40, 0))#64, sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00))#65, sum(coalesce(cast(cs_net_paid#41 as decimal(12,2)), 0.00))#66] +Results [3]: [cs_item_sk#38 AS item#67, (cast(sum(coalesce(cr_return_quantity#47, 0))#63 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#40, 0))#64 as decimal(15,4))) AS return_ratio#68, (cast(sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00))#65 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#41 as decimal(12,2)), 0.00))#66 as decimal(15,4))) AS currency_ratio#69] + +(43) Exchange +Input [3]: [item#67, return_ratio#68, currency_ratio#69] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(41) Sort [codegen id : 10] -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 +(44) Sort [codegen id : 8] +Input [3]: [item#67, return_ratio#68, currency_ratio#69] +Arguments: [return_ratio#68 ASC NULLS FIRST], false, 0 -(42) Window -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] +(45) Window +Input [3]: [item#67, return_ratio#68, currency_ratio#69] +Arguments: [rank(return_ratio#68) windowspecdefinition(return_ratio#68 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#70], [return_ratio#68 ASC NULLS FIRST] -(43) Sort [codegen id : 11] -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 9] +Input [4]: [item#67, return_ratio#68, currency_ratio#69, return_rank#70] +Arguments: [currency_ratio#69 ASC NULLS FIRST], false, 0 -(44) Window -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] +(47) Window +Input [4]: [item#67, return_ratio#68, currency_ratio#69, return_rank#70] +Arguments: [rank(currency_ratio#69) windowspecdefinition(currency_ratio#69 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#71], [currency_ratio#69 ASC NULLS FIRST] -(45) Filter [codegen id : 12] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] -Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) +(48) Filter [codegen id : 10] +Input [5]: [item#67, return_ratio#68, currency_ratio#69, return_rank#70, currency_rank#71] +Condition : ((return_rank#70 <= 10) OR (currency_rank#71 <= 10)) -(46) Project [codegen id : 12] -Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +(49) Project [codegen id : 10] +Output [5]: [catalog AS channel#72, item#67, return_ratio#68, return_rank#70, currency_rank#71] +Input [5]: [item#67, return_ratio#68, currency_ratio#69, return_rank#70, currency_rank#71] -(47) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +(50) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_net_profit#77, ss_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(48) CometFilter -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) +(51) CometFilter +Input [6]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_net_profit#77, ss_sold_date_sk#78] +Condition : (((((((isnotnull(ss_net_profit#77) AND isnotnull(ss_net_paid#76)) AND isnotnull(ss_quantity#75)) AND (ss_net_profit#77 > 1.00)) AND (ss_net_paid#76 > 0.00)) AND (ss_quantity#75 > 0)) AND isnotnull(ss_ticket_number#74)) AND isnotnull(ss_item_sk#73)) -(49) CometProject -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(52) CometProject +Input [6]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_net_profit#77, ss_sold_date_sk#78] +Arguments: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78], [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78] -(50) CometBroadcastExchange -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(53) CometBroadcastExchange +Input [5]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78] +Arguments: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78] -(51) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +(54) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(52) CometFilter -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) - -(53) CometProject -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(54) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner - -(55) CometProject -Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] - -(56) ColumnarToRow [codegen id : 14] -Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] - -(57) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#83] - -(58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#76] -Right keys [1]: [d_date_sk#83] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 14] -Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] - -(60) HashAggregate [codegen id : 14] -Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Keys [1]: [ss_item_sk#71] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] -Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(61) Exchange -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(62) HashAggregate [codegen id : 15] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Keys [1]: [ss_item_sk#71] -Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] -Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] - -(63) Exchange -Input [3]: [item#100, return_ratio#101, currency_ratio#102] +(55) CometFilter +Input [5]: [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83, sr_returned_date_sk#84] +Condition : (((isnotnull(sr_return_amt#83) AND (sr_return_amt#83 > 10000.00)) AND isnotnull(sr_ticket_number#81)) AND isnotnull(sr_item_sk#80)) + +(56) CometProject +Input [5]: [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83, sr_returned_date_sk#84] +Arguments: [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83], [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83] + +(57) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78] +Right output [4]: [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83] +Arguments: [ss_ticket_number#74, ss_item_sk#73], [sr_ticket_number#81, sr_item_sk#80], Inner, BuildLeft + +(58) CometProject +Input [9]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83] +Arguments: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_return_quantity#82, sr_return_amt#83], [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_return_quantity#82, sr_return_amt#83] + +(59) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#85] + +(60) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_return_quantity#82, sr_return_amt#83] +Right output [1]: [d_date_sk#85] +Arguments: [ss_sold_date_sk#78], [d_date_sk#85], Inner, BuildRight + +(61) CometProject +Input [7]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_return_quantity#82, sr_return_amt#83, d_date_sk#85] +Arguments: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83], [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83] + +(62) ColumnarToRow [codegen id : 11] +Input [5]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83] + +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83] +Keys [1]: [ss_item_sk#73] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#82, 0)), partial_sum(coalesce(ss_quantity#75, 0)), partial_sum(coalesce(cast(sr_return_amt#83 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#76 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#86, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] +Results [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] + +(64) Exchange +Input [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Arguments: hashpartitioning(ss_item_sk#73, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(65) HashAggregate [codegen id : 12] +Input [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Keys [1]: [ss_item_sk#73] +Functions [4]: [sum(coalesce(sr_return_quantity#82, 0)), sum(coalesce(ss_quantity#75, 0)), sum(coalesce(cast(sr_return_amt#83 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#76 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#82, 0))#98, sum(coalesce(ss_quantity#75, 0))#99, sum(coalesce(cast(sr_return_amt#83 as decimal(12,2)), 0.00))#100, sum(coalesce(cast(ss_net_paid#76 as decimal(12,2)), 0.00))#101] +Results [3]: [ss_item_sk#73 AS item#102, (cast(sum(coalesce(sr_return_quantity#82, 0))#98 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#75, 0))#99 as decimal(15,4))) AS return_ratio#103, (cast(sum(coalesce(cast(sr_return_amt#83 as decimal(12,2)), 0.00))#100 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#76 as decimal(12,2)), 0.00))#101 as decimal(15,4))) AS currency_ratio#104] + +(66) Exchange +Input [3]: [item#102, return_ratio#103, currency_ratio#104] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(64) Sort [codegen id : 16] -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 +(67) Sort [codegen id : 13] +Input [3]: [item#102, return_ratio#103, currency_ratio#104] +Arguments: [return_ratio#103 ASC NULLS FIRST], false, 0 -(65) Window -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] +(68) Window +Input [3]: [item#102, return_ratio#103, currency_ratio#104] +Arguments: [rank(return_ratio#103) windowspecdefinition(return_ratio#103 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#105], [return_ratio#103 ASC NULLS FIRST] -(66) Sort [codegen id : 17] -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 +(69) Sort [codegen id : 14] +Input [4]: [item#102, return_ratio#103, currency_ratio#104, return_rank#105] +Arguments: [currency_ratio#104 ASC NULLS FIRST], false, 0 -(67) Window -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] +(70) Window +Input [4]: [item#102, return_ratio#103, currency_ratio#104, return_rank#105] +Arguments: [rank(currency_ratio#104) windowspecdefinition(currency_ratio#104 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#106], [currency_ratio#104 ASC NULLS FIRST] -(68) Filter [codegen id : 18] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) +(71) Filter [codegen id : 15] +Input [5]: [item#102, return_ratio#103, currency_ratio#104, return_rank#105, currency_rank#106] +Condition : ((return_rank#105 <= 10) OR (currency_rank#106 <= 10)) -(69) Project [codegen id : 18] -Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] +(72) Project [codegen id : 15] +Output [5]: [store AS channel#107, item#102, return_ratio#103, return_rank#105, currency_rank#106] +Input [5]: [item#102, return_ratio#103, currency_ratio#104, return_rank#105, currency_rank#106] -(70) Union +(73) Union -(71) HashAggregate [codegen id : 19] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(74) HashAggregate [codegen id : 16] +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Keys [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] -(72) Exchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(75) Exchange +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Arguments: hashpartitioning(channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(73) HashAggregate [codegen id : 20] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(76) HashAggregate [codegen id : 17] +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Keys [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] -(74) TakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(77) TakeOrderedAndProject +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Arguments: 100, [channel#37 ASC NULLS FIRST, return_rank#35 ASC NULLS FIRST, currency_rank#36 ASC NULLS FIRST], [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan parquet spark_catalog.default.date_dim (75) +BroadcastExchange (82) ++- * ColumnarToRow (81) + +- CometProject (80) + +- CometFilter (79) + +- CometScan parquet spark_catalog.default.date_dim (78) -(75) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#106, d_moy#107] +(78) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] -Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) +(79) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(77) CometProject -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +(80) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(78) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(79) BroadcastExchange +(82) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt index 43ebf34cc0..d1480ef10b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt @@ -1,35 +1,35 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (20) + WholeStageCodegen (17) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (19) + WholeStageCodegen (16) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (5) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (5) + WholeStageCodegen (4) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (4) + WholeStageCodegen (3) Sort [return_ratio] InputAdapter Exchange #2 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ws_item_sk] #3 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] CometBroadcastExchange #4 @@ -47,34 +47,36 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_return_amt,wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (12) + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (11) + WholeStageCodegen (9) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (10) + WholeStageCodegen (8) Sort [return_ratio] InputAdapter - Exchange #6 - WholeStageCodegen (9) + Exchange #7 + WholeStageCodegen (7) HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [cs_item_sk] #7 - WholeStageCodegen (8) + Exchange [cs_item_sk] #8 + WholeStageCodegen (6) HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #8 + CometBroadcastExchange #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] @@ -82,34 +84,33 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_return_amount,cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (18) + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (15) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (17) + WholeStageCodegen (14) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (16) + WholeStageCodegen (13) Sort [return_ratio] InputAdapter - Exchange #9 - WholeStageCodegen (15) + Exchange #10 + WholeStageCodegen (12) HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (14) + Exchange [ss_item_sk] #11 + WholeStageCodegen (11) HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #11 + CometBroadcastExchange #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] @@ -117,5 +118,4 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt index e3f7538d1c..e6d8368924 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt @@ -5,68 +5,68 @@ TakeOrderedAndProject (70) +- * HashAggregate (67) +- * Expand (66) +- Union (65) - :- * HashAggregate (20) - : +- Exchange (19) - : +- * HashAggregate (18) - : +- * Project (17) - : +- * BroadcastHashJoin Inner BuildRight (16) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (8) - : : : +- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : +- ReusedExchange (9) - : +- BroadcastExchange (15) - : +- * ColumnarToRow (14) - : +- CometFilter (13) - : +- CometScan parquet spark_catalog.default.store (12) - :- * HashAggregate (40) - : +- Exchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- CometUnion (27) - : : : :- CometProject (23) - : : : : +- CometFilter (22) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (24) - : : +- ReusedExchange (29) - : +- BroadcastExchange (35) - : +- * ColumnarToRow (34) - : +- CometFilter (33) - : +- CometScan parquet spark_catalog.default.catalog_page (32) + :- * HashAggregate (22) + : +- Exchange (21) + : +- * HashAggregate (20) + : +- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometUnion (7) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : +- CometBroadcastExchange (11) + : : +- CometProject (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.date_dim (8) + : +- CometBroadcastExchange (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.store (14) + :- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * ColumnarToRow (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometUnion (29) + : : : :- CometProject (25) + : : : : +- CometFilter (24) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (23) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) + : : +- ReusedExchange (30) + : +- CometBroadcastExchange (35) + : +- CometFilter (34) + : +- CometScan parquet spark_catalog.default.catalog_page (33) +- * HashAggregate (64) +- Exchange (63) +- * HashAggregate (62) - +- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * Project (55) - : +- * BroadcastHashJoin Inner BuildRight (54) - : :- * ColumnarToRow (52) - : : +- CometUnion (51) - : : :- CometProject (43) - : : : +- CometFilter (42) - : : : +- CometScan parquet spark_catalog.default.web_sales (41) - : : +- CometProject (50) - : : +- CometBroadcastHashJoin (49) - : : :- CometBroadcastExchange (45) - : : : +- CometScan parquet spark_catalog.default.web_returns (44) - : : +- CometProject (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.web_sales (46) - : +- ReusedExchange (53) - +- BroadcastExchange (59) - +- * ColumnarToRow (58) + +- * ColumnarToRow (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometUnion (52) + : : :- CometProject (44) + : : : +- CometFilter (43) + : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometBroadcastExchange (46) + : : : +- CometScan parquet spark_catalog.default.web_returns (45) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan parquet spark_catalog.default.web_sales (47) + : +- ReusedExchange (53) + +- CometBroadcastExchange (58) +- CometFilter (57) +- CometScan parquet spark_catalog.default.web_site (56) @@ -107,310 +107,311 @@ Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, n Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] -(8) ColumnarToRow [codegen id : 3] -Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] +ReadSchema: struct + +(9) CometFilter +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(9) ReusedExchange [Reuses operator id: 75] -Output [1]: [d_date_sk#22] +(10) CometProject +Input [2]: [d_date_sk#22, d_date#23] +Arguments: [d_date_sk#22], [d_date_sk#22] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: [d_date_sk#22] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [1]: [d_date_sk#22] +Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight -(11) Project [codegen id : 3] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +(13) CometProject Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] +Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -(12) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] +(14) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_store_id#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(13) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) +(15) CometFilter +Input [2]: [s_store_sk#24, s_store_id#25] +Condition : isnotnull(s_store_sk#24) -(14) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#23, s_store_id#24] +(16) CometBroadcastExchange +Input [2]: [s_store_sk#24, s_store_id#25] +Arguments: [s_store_sk#24, s_store_id#25] -(15) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(17) CometBroadcastHashJoin +Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [2]: [s_store_sk#24, s_store_id#25] +Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None +(18) CometProject +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] +Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -(17) Project [codegen id : 3] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#24] +(19) ColumnarToRow [codegen id : 1] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -(18) HashAggregate [codegen id : 3] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Keys [1]: [s_store_id#24] +(20) HashAggregate [codegen id : 1] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] -Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] +Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(19) Exchange -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(21) Exchange +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Keys [1]: [s_store_id#24] +(22) HashAggregate [codegen id : 2] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#39, store channel AS channel#40, concat(store, s_store_id#24) AS id#41] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#38, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#39, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#40, store channel AS channel#41, concat(store, s_store_id#25) AS id#42] -(21) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(23) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#47)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(22) CometFilter -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : isnotnull(cs_catalog_page_sk#42) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : isnotnull(cs_catalog_page_sk#43) -(23) CometProject -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52], [cs_catalog_page_sk#42 AS page_sk#47, cs_sold_date_sk#45 AS date_sk#48, cs_ext_sales_price#43 AS sales_price#49, cs_net_profit#44 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] +(25) CometProject +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53], [cs_catalog_page_sk#43 AS page_sk#48, cs_sold_date_sk#46 AS date_sk#49, cs_ext_sales_price#44 AS sales_price#50, cs_net_profit#45 AS profit#51, 0.00 AS return_amt#52, 0.00 AS net_loss#53] -(24) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +(26) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#47)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Condition : isnotnull(cr_catalog_page_sk#53) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Condition : isnotnull(cr_catalog_page_sk#54) -(26) CometProject -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Arguments: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62], [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] +(28) CometProject +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Arguments: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63], [cr_catalog_page_sk#54 AS page_sk#58, cr_returned_date_sk#57 AS date_sk#59, 0.00 AS sales_price#60, 0.00 AS profit#61, cr_return_amount#55 AS return_amt#62, cr_net_loss#56 AS net_loss#63] -(27) CometUnion -Child 0 Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] -Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62] +(29) CometUnion +Child 0 Input [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] +Child 1 Input [6]: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63] -(28) ColumnarToRow [codegen id : 7] -Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#64] -(29) ReusedExchange [Reuses operator id: 75] -Output [1]: [d_date_sk#63] +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] +Right output [1]: [d_date_sk#64] +Arguments: [date_sk#49], [d_date_sk#64], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [date_sk#48] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None +(32) CometProject +Input [7]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53, d_date_sk#64] +Arguments: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53], [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53] -(31) Project [codegen id : 7] -Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] -Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] - -(32) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +(33) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(33) CometFilter -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) - -(34) ColumnarToRow [codegen id : 6] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] - -(35) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 7] -Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] -Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#65] - -(38) HashAggregate [codegen id : 7] -Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] -Keys [1]: [cp_catalog_page_id#65] -Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum#66, sum#67, sum#68, sum#69] -Results [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] - -(39) Exchange -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(40) HashAggregate [codegen id : 8] -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Keys [1]: [cp_catalog_page_id#65] -Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#49))#74,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(return_amt#51))#75,17,2) AS returns#79, (MakeDecimal(sum(UnscaledValue(profit#50))#76,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#77,17,2)) AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#65) AS id#82] - -(41) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +(34) CometFilter +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Condition : isnotnull(cp_catalog_page_sk#65) + +(35) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Arguments: [cp_catalog_page_sk#65, cp_catalog_page_id#66] + +(36) CometBroadcastHashJoin +Left output [5]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53] +Right output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Arguments: [page_sk#48], [cp_catalog_page_sk#65], Inner, BuildRight + +(37) CometProject +Input [7]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_sk#65, cp_catalog_page_id#66] +Arguments: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66], [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] + +(38) ColumnarToRow [codegen id : 3] +Input [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] + +(39) HashAggregate [codegen id : 3] +Input [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [partial_sum(UnscaledValue(sales_price#50)), partial_sum(UnscaledValue(return_amt#52)), partial_sum(UnscaledValue(profit#51)), partial_sum(UnscaledValue(net_loss#53))] +Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] +Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] + +(40) Exchange +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(41) HashAggregate [codegen id : 4] +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [sum(UnscaledValue(sales_price#50)), sum(UnscaledValue(return_amt#52)), sum(UnscaledValue(profit#51)), sum(UnscaledValue(net_loss#53))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#50))#75, sum(UnscaledValue(return_amt#52))#76, sum(UnscaledValue(profit#51))#77, sum(UnscaledValue(net_loss#53))#78] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#50))#75,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#52))#76,17,2) AS returns#80, (MakeDecimal(sum(UnscaledValue(profit#51))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#53))#78,17,2)) AS profit#81, catalog channel AS channel#82, concat(catalog_page, cp_catalog_page_id#66) AS id#83] + +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(42) CometFilter -Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] -Condition : isnotnull(ws_web_site_sk#83) +(43) CometFilter +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] +Condition : isnotnull(ws_web_site_sk#84) -(43) CometProject -Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] -Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] +(44) CometProject +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] +Arguments: [wsr_web_site_sk#89, date_sk#90, sales_price#91, profit#92, return_amt#93, net_loss#94], [ws_web_site_sk#84 AS wsr_web_site_sk#89, ws_sold_date_sk#87 AS date_sk#90, ws_ext_sales_price#85 AS sales_price#91, ws_net_profit#86 AS profit#92, 0.00 AS return_amt#93, 0.00 AS net_loss#94] -(44) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +(45) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(wr_returned_date_sk#99), dynamicpruningexpression(wr_returned_date_sk#99 IN dynamicpruning#88)] ReadSchema: struct -(45) CometBroadcastExchange -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] +Arguments: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] -(46) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +(47) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(47) CometFilter -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) - -(48) CometProject -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +(48) CometFilter +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] +Condition : ((isnotnull(ws_item_sk#100) AND isnotnull(ws_order_number#102)) AND isnotnull(ws_web_site_sk#101)) -(49) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner +(49) CometProject +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] +Arguments: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102], [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] -(50) CometProject -Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] +Right output [3]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Arguments: [wr_item_sk#95, wr_order_number#96], [ws_item_sk#100, ws_order_number#102], Inner, BuildLeft -(51) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] +(51) CometProject +Input [8]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99, ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Arguments: [wsr_web_site_sk#104, date_sk#105, sales_price#106, profit#107, return_amt#108, net_loss#109], [ws_web_site_sk#101 AS wsr_web_site_sk#104, wr_returned_date_sk#99 AS date_sk#105, 0.00 AS sales_price#106, 0.00 AS profit#107, wr_return_amt#97 AS return_amt#108, wr_net_loss#98 AS net_loss#109] -(52) ColumnarToRow [codegen id : 11] -Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#89, date_sk#90, sales_price#91, profit#92, return_amt#93, net_loss#94] +Child 1 Input [6]: [wsr_web_site_sk#104, date_sk#105, sales_price#106, profit#107, return_amt#108, net_loss#109] -(53) ReusedExchange [Reuses operator id: 75] -Output [1]: [d_date_sk#109] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#110] -(54) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#89] -Right keys [1]: [d_date_sk#109] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#89, date_sk#90, sales_price#91, profit#92, return_amt#93, net_loss#94] +Right output [1]: [d_date_sk#110] +Arguments: [date_sk#90], [d_date_sk#110], Inner, BuildRight -(55) Project [codegen id : 11] -Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] -Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] +(55) CometProject +Input [7]: [wsr_web_site_sk#89, date_sk#90, sales_price#91, profit#92, return_amt#93, net_loss#94, d_date_sk#110] +Arguments: [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94], [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94] (56) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#110, web_site_id#111] +Output [2]: [web_site_sk#111, web_site_id#112] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (57) CometFilter -Input [2]: [web_site_sk#110, web_site_id#111] -Condition : isnotnull(web_site_sk#110) +Input [2]: [web_site_sk#111, web_site_id#112] +Condition : isnotnull(web_site_sk#111) -(58) ColumnarToRow [codegen id : 10] -Input [2]: [web_site_sk#110, web_site_id#111] +(58) CometBroadcastExchange +Input [2]: [web_site_sk#111, web_site_id#112] +Arguments: [web_site_sk#111, web_site_id#112] -(59) BroadcastExchange -Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(59) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94] +Right output [2]: [web_site_sk#111, web_site_id#112] +Arguments: [wsr_web_site_sk#89], [web_site_sk#111], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [wsr_web_site_sk#88] -Right keys [1]: [web_site_sk#110] -Join type: Inner -Join condition: None +(60) CometProject +Input [7]: [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_sk#111, web_site_id#112] +Arguments: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112], [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] -(61) Project [codegen id : 11] -Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] -Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] +(61) ColumnarToRow [codegen id : 5] +Input [5]: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] -(62) HashAggregate [codegen id : 11] -Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] -Keys [1]: [web_site_id#111] -Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] -Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] +(62) HashAggregate [codegen id : 5] +Input [5]: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] +Keys [1]: [web_site_id#112] +Functions [4]: [partial_sum(UnscaledValue(sales_price#91)), partial_sum(UnscaledValue(return_amt#93)), partial_sum(UnscaledValue(profit#92)), partial_sum(UnscaledValue(net_loss#94))] +Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] +Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] (63) Exchange -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(64) HashAggregate [codegen id : 12] -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Keys [1]: [web_site_id#111] -Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#126, web channel AS channel#127, concat(web_site, web_site_id#111) AS id#128] +(64) HashAggregate [codegen id : 6] +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Keys [1]: [web_site_id#112] +Functions [4]: [sum(UnscaledValue(sales_price#91)), sum(UnscaledValue(return_amt#93)), sum(UnscaledValue(profit#92)), sum(UnscaledValue(net_loss#94))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#91))#121, sum(UnscaledValue(return_amt#93))#122, sum(UnscaledValue(profit#92))#123, sum(UnscaledValue(net_loss#94))#124] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#91))#121,17,2) AS sales#125, MakeDecimal(sum(UnscaledValue(return_amt#93))#122,17,2) AS returns#126, (MakeDecimal(sum(UnscaledValue(profit#92))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#94))#124,17,2)) AS profit#127, web channel AS channel#128, concat(web_site, web_site_id#112) AS id#129] (65) Union -(66) Expand [codegen id : 13] -Input [5]: [sales#37, returns#38, profit#39, channel#40, id#41] -Arguments: [[sales#37, returns#38, profit#39, channel#40, id#41, 0], [sales#37, returns#38, profit#39, channel#40, null, 1], [sales#37, returns#38, profit#39, null, null, 3]], [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] +(66) Expand [codegen id : 7] +Input [5]: [sales#38, returns#39, profit#40, channel#41, id#42] +Arguments: [[sales#38, returns#39, profit#40, channel#41, id#42, 0], [sales#38, returns#39, profit#40, channel#41, null, 1], [sales#38, returns#39, profit#40, null, null, 3]], [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] -(67) HashAggregate [codegen id : 13] -Input [6]: [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] -Keys [3]: [channel#129, id#130, spark_grouping_id#131] -Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -Results [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +(67) HashAggregate [codegen id : 7] +Input [6]: [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] +Keys [3]: [channel#130, id#131, spark_grouping_id#132] +Functions [3]: [partial_sum(sales#38), partial_sum(returns#39), partial_sum(profit#40)] +Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Results [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] (68) Exchange -Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#129, id#130, spark_grouping_id#131, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Arguments: hashpartitioning(channel#130, id#131, spark_grouping_id#132, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(69) HashAggregate [codegen id : 14] -Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [3]: [channel#129, id#130, spark_grouping_id#131] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#144, sum(returns#38)#145, sum(profit#39)#146] -Results [5]: [channel#129, id#130, sum(sales#37)#144 AS sales#147, sum(returns#38)#145 AS returns#148, sum(profit#39)#146 AS profit#149] +(69) HashAggregate [codegen id : 8] +Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Keys [3]: [channel#130, id#131, spark_grouping_id#132] +Functions [3]: [sum(sales#38), sum(returns#39), sum(profit#40)] +Aggregate Attributes [3]: [sum(sales#38)#145, sum(returns#39)#146, sum(profit#40)#147] +Results [5]: [channel#130, id#131, sum(sales#38)#145 AS sales#148, sum(returns#39)#146 AS returns#149, sum(profit#40)#147 AS profit#150] (70) TakeOrderedAndProject -Input [5]: [channel#129, id#130, sales#147, returns#148, profit#149] -Arguments: 100, [channel#129 ASC NULLS FIRST, id#130 ASC NULLS FIRST], [channel#129, id#130, sales#147, returns#148, profit#149] +Input [5]: [channel#130, id#131, sales#148, returns#149, profit#150] +Arguments: 100, [channel#130 ASC NULLS FIRST, id#131 ASC NULLS FIRST], [channel#130, id#131, sales#148, returns#149, profit#150] ===== Subqueries ===== @@ -423,18 +424,18 @@ BroadcastExchange (75) (71) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#150] +Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct (72) CometFilter -Input [2]: [d_date_sk#22, d_date#150] -Condition : (((isnotnull(d_date#150) AND (d_date#150 >= 2000-08-23)) AND (d_date#150 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) (73) CometProject -Input [2]: [d_date_sk#22, d_date#150] +Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] (74) ColumnarToRow [codegen id : 1] @@ -442,16 +443,16 @@ Input [1]: [d_date_sk#22] (75) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#99 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt index 3d539d5917..3dc3cca6ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (14) + WholeStageCodegen (8) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (13) + WholeStageCodegen (7) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [s_store_id] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,s_store_sk] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [ss_store_sk] @@ -36,27 +36,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (8) + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #5 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter - Exchange [cp_catalog_page_id] #5 - WholeStageCodegen (7) + Exchange [cp_catalog_page_id] #6 + WholeStageCodegen (3) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [cs_catalog_page_sk] @@ -66,27 +64,22 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [cr_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (12) + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #7 + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter - Exchange [web_site_id] #7 - WholeStageCodegen (11) + Exchange [web_site_id] #8 + WholeStageCodegen (5) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [ws_web_site_sk] @@ -94,18 +87,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #8 + CometBroadcastExchange #9 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #10 + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt index 0182e0ac1e..ff35888eef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt @@ -3,30 +3,30 @@ TakeOrderedAndProject (28) +- * HashAggregate (27) +- Exchange (26) +- * HashAggregate (25) - +- * Project (24) - +- * BroadcastHashJoin Inner BuildRight (23) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.date_dim (16) - +- ReusedExchange (22) + +- * ColumnarToRow (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometProject (17) + : +- CometBroadcastHashJoin (16) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.store_returns (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.store (8) + : +- CometBroadcastExchange (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.date_dim (13) + +- CometBroadcastExchange (21) + +- CometProject (20) + +- CometFilter (19) + +- CometScan parquet spark_catalog.default.date_dim (18) (1) Scan parquet spark_catalog.default.store_sales @@ -41,10 +41,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Arguments: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -(7) BroadcastExchange -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(input[2, int, false], input[0, int, false], input[1, int, false]),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Right output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Arguments: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2], [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 5] -Left keys [3]: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 5] -Output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] +(7) CometProject Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Arguments: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9], [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] -(10) Scan parquet spark_catalog.default.store +(8) Scan parquet spark_catalog.default.store Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Condition : isnotnull(s_store_sk#11) -(12) ColumnarToRow [codegen id : 2] -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] +Right output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight -(15) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +(12) CometProject Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -(16) Scan parquet spark_catalog.default.date_dim +(13) Scan parquet spark_catalog.default.date_dim Output [1]: [d_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [1]: [d_date_sk#22] Condition : isnotnull(d_date_sk#22) -(18) ColumnarToRow [codegen id : 3] -Input [1]: [d_date_sk#22] - -(19) BroadcastExchange +(15) CometBroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [d_date_sk#22] -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Right output [1]: [d_date_sk#22] +Arguments: [ss_sold_date_sk#5], [d_date_sk#22], Inner, BuildRight -(21) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +(17) CometProject Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#22] +Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -(22) ReusedExchange [Reuses operator id: 33] -Output [1]: [d_date_sk#23] +(18) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#23, d_year#24, d_moy#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] +ReadSchema: struct -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_returned_date_sk#9] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None +(19) CometFilter +Input [3]: [d_date_sk#23, d_year#24, d_moy#25] +Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2001)) AND (d_moy#25 = 8)) AND isnotnull(d_date_sk#23)) -(24) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +(20) CometProject +Input [3]: [d_date_sk#23, d_year#24, d_moy#25] +Arguments: [d_date_sk#23], [d_date_sk#23] + +(21) CometBroadcastExchange +Input [1]: [d_date_sk#23] +Arguments: [d_date_sk#23] + +(22) CometBroadcastHashJoin +Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Right output [1]: [d_date_sk#23] +Arguments: [sr_returned_date_sk#9], [d_date_sk#23], Inner, BuildRight + +(23) CometProject Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#23] +Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] + +(24) ColumnarToRow [codegen id : 1] +Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -(25) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 1] Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum#24, sum#25, sum#26, sum#27, sum#28] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] +Aggregate Attributes [5]: [sum#26, sum#27, sum#28, sum#29, sum#30] +Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#31, sum#32, sum#33, sum#34, sum#35] (26) Exchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#31, sum#32, sum#33, sum#34, sum#35] +Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(27) HashAggregate [codegen id : 6] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] +(27) HashAggregate [codegen id : 2] +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#31, sum#32, sum#33, sum#34, sum#35] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34 AS 30 days #39, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35 AS 31 - 60 days #40, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36 AS 61 - 90 days #41, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37 AS 91 - 120 days #42, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38 AS >120 days #43] +Aggregate Attributes [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#36, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#37, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#38, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#39, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#40] +Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#36 AS 30 days #41, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#37 AS 31 - 60 days #42, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#38 AS 61 - 90 days #43, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#39 AS 91 - 120 days #44, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#40 AS >120 days #45] (28) TakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] -Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #41, 31 - 60 days #42, 61 - 90 days #43, 91 - 120 days #44, >120 days #45] +Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #41, 31 - 60 days #42, 61 - 90 days #43, 91 - 120 days #44, >120 days #45] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 +Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (33) +- * ColumnarToRow (32) +- CometProject (31) @@ -175,18 +178,18 @@ BroadcastExchange (33) (29) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#23, d_year#44, d_moy#45] +Output [3]: [d_date_sk#23, d_year#24, d_moy#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] ReadSchema: struct (30) CometFilter -Input [3]: [d_date_sk#23, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 8)) AND isnotnull(d_date_sk#23)) +Input [3]: [d_date_sk#23, d_year#24, d_moy#25] +Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2001)) AND (d_moy#25 = 8)) AND isnotnull(d_date_sk#23)) (31) CometProject -Input [3]: [d_date_sk#23, d_year#44, d_moy#45] +Input [3]: [d_date_sk#23, d_year#24, d_moy#25] Arguments: [d_date_sk#23], [d_date_sk#23] (32) ColumnarToRow [codegen id : 1] @@ -194,6 +197,6 @@ Input [1]: [d_date_sk#23] (33) BroadcastExchange Input [1]: [d_date_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt index dfdcaf4975..cd8a8ff7f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt @@ -1,50 +1,40 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter Exchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #2 + CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #4 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometBroadcastExchange #5 + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt index 2613551f0c..20fdfd4a6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt @@ -1,41 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (37) -+- * Filter (36) - +- Window (35) - +- * Sort (34) - +- Exchange (33) - +- * Project (32) - +- * SortMergeJoin FullOuter (31) - :- * Sort (15) - : +- Exchange (14) - : +- * Project (13) - : +- Window (12) - : +- * Sort (11) - : +- Exchange (10) - : +- * HashAggregate (9) - : +- Exchange (8) - : +- * HashAggregate (7) - : +- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- * Sort (30) - +- Exchange (29) - +- * Project (28) - +- Window (27) - +- * Sort (26) - +- Exchange (25) - +- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.store_sales (16) - +- ReusedExchange (19) +TakeOrderedAndProject (40) ++- * Filter (39) + +- Window (38) + +- * Sort (37) + +- Exchange (36) + +- * Project (35) + +- * SortMergeJoin FullOuter (34) + :- * Sort (18) + : +- Exchange (17) + : +- * Project (16) + : +- Window (15) + : +- * Sort (14) + : +- Exchange (13) + : +- * HashAggregate (12) + : +- Exchange (11) + : +- * HashAggregate (10) + : +- * ColumnarToRow (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.date_dim (3) + +- * Sort (33) + +- Exchange (32) + +- * Project (31) + +- Window (30) + +- * Sort (29) + +- Exchange (28) + +- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometFilter (20) + : +- CometScan parquet spark_catalog.default.store_sales (19) + +- ReusedExchange (21) (1) Scan parquet spark_catalog.default.web_sales @@ -50,196 +53,210 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 42] -Output [2]: [d_date_sk#5, d_date#6] +(4) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] + +(6) CometBroadcastExchange +Input [2]: [d_date_sk#5, d_date#6] +Arguments: [d_date_sk#5, d_date#6] -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +Right output [2]: [d_date_sk#5, d_date#6] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] +Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] + +(9) ColumnarToRow [codegen id : 1] +Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -(7) HashAggregate [codegen id : 2] +(10) HashAggregate [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] +Aggregate Attributes [1]: [sum#8] +Results [3]: [ws_item_sk#1, d_date#6, sum#9] -(8) Exchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] +(11) Exchange +Input [3]: [ws_item_sk#1, d_date#6, sum#9] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) HashAggregate [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] +(12) HashAggregate [codegen id : 2] +Input [3]: [ws_item_sk#1, d_date#6, sum#9] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS _w0#11, ws_item_sk#1] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#10] +Results [4]: [ws_item_sk#1 AS item_sk#11, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#10,17,2) AS _w0#12, ws_item_sk#1] -(10) Exchange -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +(13) Exchange +Input [4]: [item_sk#11, d_date#6, _w0#12, ws_item_sk#1] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) Sort [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +(14) Sort [codegen id : 3] +Input [4]: [item_sk#11, d_date#6, _w0#12, ws_item_sk#1] Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(12) Window -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(15) Window +Input [4]: [item_sk#11, d_date#6, _w0#12, ws_item_sk#1] +Arguments: [sum(_w0#12) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#13], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(13) Project [codegen id : 5] -Output [3]: [item_sk#10, d_date#6, cume_sales#12] -Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] +(16) Project [codegen id : 4] +Output [3]: [item_sk#11, d_date#6, cume_sales#13] +Input [5]: [item_sk#11, d_date#6, _w0#12, ws_item_sk#1, cume_sales#13] -(14) Exchange -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(17) Exchange +Input [3]: [item_sk#11, d_date#6, cume_sales#13] +Arguments: hashpartitioning(item_sk#11, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(15) Sort [codegen id : 6] -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 +(18) Sort [codegen id : 5] +Input [3]: [item_sk#11, d_date#6, cume_sales#13] +Arguments: [item_sk#11 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(16) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +(19) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#14, ss_sales_price#15, ss_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#17)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(17) CometFilter -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_item_sk#13) - -(18) ColumnarToRow [codegen id : 8] -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] - -(19) ReusedExchange [Reuses operator id: 42] -Output [2]: [d_date_sk#17, d_date#18] - -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 8] -Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] -Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] - -(22) HashAggregate [codegen id : 8] -Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] -Keys [2]: [ss_item_sk#13, d_date#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [ss_item_sk#13, d_date#18, sum#20] - -(23) Exchange -Input [3]: [ss_item_sk#13, d_date#18, sum#20] -Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(24) HashAggregate [codegen id : 9] -Input [3]: [ss_item_sk#13, d_date#18, sum#20] -Keys [2]: [ss_item_sk#13, d_date#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#21] -Results [4]: [ss_item_sk#13 AS item_sk#22, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#21,17,2) AS _w0#23, ss_item_sk#13] - -(25) Exchange -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(26) Sort [codegen id : 10] -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -Arguments: [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 - -(27) Window -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -Arguments: [sum(_w0#23) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#24], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] - -(28) Project [codegen id : 11] -Output [3]: [item_sk#22, d_date#18, cume_sales#24] -Input [5]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13, cume_sales#24] - -(29) Exchange -Input [3]: [item_sk#22, d_date#18, cume_sales#24] -Arguments: hashpartitioning(item_sk#22, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(30) Sort [codegen id : 12] -Input [3]: [item_sk#22, d_date#18, cume_sales#24] -Arguments: [item_sk#22 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 - -(31) SortMergeJoin [codegen id : 13] -Left keys [2]: [item_sk#10, d_date#6] -Right keys [2]: [item_sk#22, d_date#18] +(20) CometFilter +Input [3]: [ss_item_sk#14, ss_sales_price#15, ss_sold_date_sk#16] +Condition : isnotnull(ss_item_sk#14) + +(21) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#18, d_date#19] + +(22) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#14, ss_sales_price#15, ss_sold_date_sk#16] +Right output [2]: [d_date_sk#18, d_date#19] +Arguments: [ss_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight + +(23) CometProject +Input [5]: [ss_item_sk#14, ss_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_date#19] +Arguments: [ss_item_sk#14, ss_sales_price#15, d_date#19], [ss_item_sk#14, ss_sales_price#15, d_date#19] + +(24) ColumnarToRow [codegen id : 6] +Input [3]: [ss_item_sk#14, ss_sales_price#15, d_date#19] + +(25) HashAggregate [codegen id : 6] +Input [3]: [ss_item_sk#14, ss_sales_price#15, d_date#19] +Keys [2]: [ss_item_sk#14, d_date#19] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#15))] +Aggregate Attributes [1]: [sum#20] +Results [3]: [ss_item_sk#14, d_date#19, sum#21] + +(26) Exchange +Input [3]: [ss_item_sk#14, d_date#19, sum#21] +Arguments: hashpartitioning(ss_item_sk#14, d_date#19, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(27) HashAggregate [codegen id : 7] +Input [3]: [ss_item_sk#14, d_date#19, sum#21] +Keys [2]: [ss_item_sk#14, d_date#19] +Functions [1]: [sum(UnscaledValue(ss_sales_price#15))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#15))#22] +Results [4]: [ss_item_sk#14 AS item_sk#23, d_date#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#15))#22,17,2) AS _w0#24, ss_item_sk#14] + +(28) Exchange +Input [4]: [item_sk#23, d_date#19, _w0#24, ss_item_sk#14] +Arguments: hashpartitioning(ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(29) Sort [codegen id : 8] +Input [4]: [item_sk#23, d_date#19, _w0#24, ss_item_sk#14] +Arguments: [ss_item_sk#14 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 + +(30) Window +Input [4]: [item_sk#23, d_date#19, _w0#24, ss_item_sk#14] +Arguments: [sum(_w0#24) windowspecdefinition(ss_item_sk#14, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#25], [ss_item_sk#14], [d_date#19 ASC NULLS FIRST] + +(31) Project [codegen id : 9] +Output [3]: [item_sk#23, d_date#19, cume_sales#25] +Input [5]: [item_sk#23, d_date#19, _w0#24, ss_item_sk#14, cume_sales#25] + +(32) Exchange +Input [3]: [item_sk#23, d_date#19, cume_sales#25] +Arguments: hashpartitioning(item_sk#23, d_date#19, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(33) Sort [codegen id : 10] +Input [3]: [item_sk#23, d_date#19, cume_sales#25] +Arguments: [item_sk#23 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 + +(34) SortMergeJoin [codegen id : 11] +Left keys [2]: [item_sk#11, d_date#6] +Right keys [2]: [item_sk#23, d_date#19] Join type: FullOuter Join condition: None -(32) Project [codegen id : 13] -Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#22 END AS item_sk#25, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#26, cume_sales#12 AS web_sales#27, cume_sales#24 AS store_sales#28] -Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#22, d_date#18, cume_sales#24] +(35) Project [codegen id : 11] +Output [4]: [CASE WHEN isnotnull(item_sk#11) THEN item_sk#11 ELSE item_sk#23 END AS item_sk#26, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#19 END AS d_date#27, cume_sales#13 AS web_sales#28, cume_sales#25 AS store_sales#29] +Input [6]: [item_sk#11, d_date#6, cume_sales#13, item_sk#23, d_date#19, cume_sales#25] -(33) Exchange -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: hashpartitioning(item_sk#25, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(36) Exchange +Input [4]: [item_sk#26, d_date#27, web_sales#28, store_sales#29] +Arguments: hashpartitioning(item_sk#26, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(34) Sort [codegen id : 14] -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 12] +Input [4]: [item_sk#26, d_date#27, web_sales#28, store_sales#29] +Arguments: [item_sk#26 ASC NULLS FIRST, d_date#27 ASC NULLS FIRST], false, 0 -(35) Window -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: [max(web_sales#27) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#29, max(store_sales#28) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#30], [item_sk#25], [d_date#26 ASC NULLS FIRST] +(38) Window +Input [4]: [item_sk#26, d_date#27, web_sales#28, store_sales#29] +Arguments: [max(web_sales#28) windowspecdefinition(item_sk#26, d_date#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#30, max(store_sales#29) windowspecdefinition(item_sk#26, d_date#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#31], [item_sk#26], [d_date#27 ASC NULLS FIRST] -(36) Filter [codegen id : 15] -Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] -Condition : ((isnotnull(web_cumulative#29) AND isnotnull(store_cumulative#30)) AND (web_cumulative#29 > store_cumulative#30)) +(39) Filter [codegen id : 13] +Input [6]: [item_sk#26, d_date#27, web_sales#28, store_sales#29, web_cumulative#30, store_cumulative#31] +Condition : ((isnotnull(web_cumulative#30) AND isnotnull(store_cumulative#31)) AND (web_cumulative#30 > store_cumulative#31)) -(37) TakeOrderedAndProject -Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] -Arguments: 100, [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] +(40) TakeOrderedAndProject +Input [6]: [item_sk#26, d_date#27, web_sales#28, store_sales#29, web_cumulative#30, store_cumulative#31] +Arguments: 100, [item_sk#26 ASC NULLS FIRST, d_date#27 ASC NULLS FIRST], [item_sk#26, d_date#27, web_sales#28, store_sales#29, web_cumulative#30, store_cumulative#31] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (42) -+- * ColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan parquet spark_catalog.default.date_dim (38) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) -(38) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#31] +(41) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(39) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] -Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1200)) AND (d_month_seq#31 <= 1211)) AND isnotnull(d_date_sk#5)) +(42) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(40) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] +(43) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(41) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(42) BroadcastExchange +(45) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt index 181cd1b98c..9ab7f18e7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt @@ -1,38 +1,38 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (15) + WholeStageCodegen (13) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (14) + WholeStageCodegen (12) Sort [item_sk,d_date] InputAdapter Exchange [item_sk] #1 - WholeStageCodegen (13) + WholeStageCodegen (11) Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] SortMergeJoin [item_sk,d_date,item_sk,d_date] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (5) Sort [item_sk,d_date] InputAdapter Exchange [item_sk,d_date] #2 - WholeStageCodegen (5) + WholeStageCodegen (4) Project [item_sk,d_date,cume_sales] InputAdapter Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (4) + WholeStageCodegen (3) Sort [ws_item_sk,d_date] InputAdapter Exchange [ws_item_sk] #3 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] InputAdapter Exchange [ws_item_sk,d_date] #4 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -43,33 +43,34 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometProject [d_date_sk,d_date] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + CometBroadcastExchange #6 + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (10) Sort [item_sk,d_date] InputAdapter - Exchange [item_sk,d_date] #6 - WholeStageCodegen (11) + Exchange [item_sk,d_date] #7 + WholeStageCodegen (9) Project [item_sk,d_date,cume_sales] InputAdapter Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (10) + WholeStageCodegen (8) Sort [ss_item_sk,d_date] InputAdapter - Exchange [ss_item_sk] #7 - WholeStageCodegen (9) + Exchange [ss_item_sk] #8 + WholeStageCodegen (7) HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] InputAdapter - Exchange [ss_item_sk,d_date] #8 - WholeStageCodegen (8) + Exchange [ss_item_sk,d_date] #9 + WholeStageCodegen (6) HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt index 3d5317eb0c..6d4feea571 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt @@ -1,25 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (19) ++- * HashAggregate (18) + +- Exchange (17) + +- * HashAggregate (16) + +- * ColumnarToRow (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- CometBroadcastExchange (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.store_sales (4) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.item (9) (1) Scan parquet spark_catalog.default.date_dim @@ -37,10 +35,7 @@ Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) Input [3]: [d_date_sk#1, d_year#2, d_moy#3] Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] -(4) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#1, d_year#2] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -48,78 +43,73 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Condition : isnotnull(ss_item_sk#4) -(7) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [d_date_sk#1, d_year#2] +Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight -(10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +(8) CometProject Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -(11) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] 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 -(12) CometFilter +(10) CometFilter Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) -(13) CometProject +(11) CometProject Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9], [i_item_sk#7, i_brand_id#8, i_brand#9] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9] -(15) BroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(13) CometBroadcastHashJoin +Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#4] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +(14) CometProject Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] + +(15) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(18) HashAggregate [codegen id : 3] +(16) HashAggregate [codegen id : 1] Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -(19) Exchange +(17) Exchange Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(21) TakeOrderedAndProject +(19) TakeOrderedAndProject Input [4]: [d_year#2, brand_id#14, brand#15, ext_price#16] Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, ext_price#16] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt index 91fdc2f17d..d0b9da8825 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt @@ -1,31 +1,23 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) 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_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_year] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt index 335dc7fa2b..e85d73a072 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt @@ -1,32 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * Project (27) - +- * Filter (26) - +- Window (25) - +- * Sort (24) - +- Exchange (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.store (14) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * Sort (25) + +- Exchange (24) + +- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * ColumnarToRow (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- CometBroadcastExchange (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.store_sales (4) + : +- CometBroadcastExchange (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.date_dim (9) + +- CometBroadcastExchange (17) + +- CometFilter (16) + +- CometScan parquet spark_catalog.default.store (15) (1) Scan parquet spark_catalog.default.item @@ -44,10 +45,7 @@ Condition : ((((i_category#4 IN (Books Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] -(4) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#1, i_manufact_id#5] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] @@ -55,140 +53,150 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) -(7) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [i_item_sk#1, i_manufact_id#5] +Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight -(10) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +(8) CometProject Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] + +(9) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(10) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] +Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) + +(11) CometProject +Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] +Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(11) ReusedExchange [Reuses operator id: 33] -Output [2]: [d_date_sk#15, d_qoy#16] +(12) CometBroadcastExchange +Input [2]: [d_date_sk#15, d_qoy#17] +Arguments: [d_date_sk#15, d_qoy#17] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Right output [2]: [d_date_sk#15, d_qoy#17] +Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(13) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16] -Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#16] +(14) CometProject +Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#17] +Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] -(14) Scan parquet spark_catalog.default.store -Output [1]: [s_store_sk#17] +(15) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#18] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [1]: [s_store_sk#17] -Condition : isnotnull(s_store_sk#17) +(16) CometFilter +Input [1]: [s_store_sk#18] +Condition : isnotnull(s_store_sk#18) -(16) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#17] +(17) CometBroadcastExchange +Input [1]: [s_store_sk#18] +Arguments: [s_store_sk#18] -(17) BroadcastExchange -Input [1]: [s_store_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(18) CometBroadcastHashJoin +Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] +Right output [1]: [s_store_sk#18] +Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#17] -Join type: Inner -Join condition: None +(19) CometProject +Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17, s_store_sk#18] +Arguments: [i_manufact_id#5, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_sales_price#12, d_qoy#17] -(19) Project [codegen id : 4] -Output [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] -Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16, s_store_sk#17] +(20) ColumnarToRow [codegen id : 1] +Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#17] -(20) HashAggregate [codegen id : 4] -Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] -Keys [2]: [i_manufact_id#5, d_qoy#16] +(21) HashAggregate [codegen id : 1] +Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#17] +Keys [2]: [i_manufact_id#5, d_qoy#17] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [i_manufact_id#5, d_qoy#16, sum#19] +Aggregate Attributes [1]: [sum#19] +Results [3]: [i_manufact_id#5, d_qoy#17, sum#20] -(21) Exchange -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) Exchange +Input [3]: [i_manufact_id#5, d_qoy#17, sum#20] +Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Keys [2]: [i_manufact_id#5, d_qoy#16] +(23) HashAggregate [codegen id : 2] +Input [3]: [i_manufact_id#5, d_qoy#17, sum#20] +Keys [2]: [i_manufact_id#5, d_qoy#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#21] +Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#21,17,2) AS sum_sales#22, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#21,17,2) AS _w0#23] -(23) Exchange -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(24) Exchange +Input [3]: [i_manufact_id#5, sum_sales#22, _w0#23] +Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(24) Sort [codegen id : 6] -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +(25) Sort [codegen id : 3] +Input [3]: [i_manufact_id#5, sum_sales#22, _w0#23] Arguments: [i_manufact_id#5 ASC NULLS FIRST], false, 0 -(25) Window -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] +(26) Window +Input [3]: [i_manufact_id#5, sum_sales#22, _w0#23] +Arguments: [avg(_w0#23) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#24], [i_manufact_id#5] -(26) Filter [codegen id : 7] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END +(27) Filter [codegen id : 4] +Input [4]: [i_manufact_id#5, sum_sales#22, _w0#23, avg_quarterly_sales#24] +Condition : CASE WHEN (avg_quarterly_sales#24 > 0.000000) THEN ((abs((sum_sales#22 - avg_quarterly_sales#24)) / avg_quarterly_sales#24) > 0.1000000000000000) ELSE false END -(27) Project [codegen id : 7] -Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] +(28) Project [codegen id : 4] +Output [3]: [i_manufact_id#5, sum_sales#22, avg_quarterly_sales#24] +Input [4]: [i_manufact_id#5, sum_sales#22, _w0#23, avg_quarterly_sales#24] -(28) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +(29) TakeOrderedAndProject +Input [3]: [i_manufact_id#5, sum_sales#22, avg_quarterly_sales#24] +Arguments: 100, [avg_quarterly_sales#24 ASC NULLS FIRST, sum_sales#22 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#22, avg_quarterly_sales#24] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) -(29) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] +(30) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(30) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) +(31) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] +Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(31) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] +(32) CometProject +Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] +Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(32) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_qoy#16] +(33) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#15, d_qoy#17] -(33) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(34) BroadcastExchange +Input [2]: [d_date_sk#15, d_qoy#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt index adda5c34f7..a047774124 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt @@ -1,51 +1,45 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_manufact_id] InputAdapter Exchange [i_manufact_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_manufact_id,d_qoy] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - Project [i_manufact_id,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometProject [i_item_sk,i_manufact_id] CometFilter [i_category,i_class,i_brand,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_qoy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt index ca308b19a4..953223fd9c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt @@ -1,59 +1,60 @@ == Physical Plan == -TakeOrderedAndProject (55) -+- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (33) - : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : :- * HashAggregate (27) - : : : : +- Exchange (26) - : : : : +- * HashAggregate (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * ColumnarToRow (8) - : : : : : : : +- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (4) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- CometProject (11) - : : : : : : +- CometFilter (10) - : : : : : : +- CometScan parquet spark_catalog.default.item (9) - : : : : : +- ReusedExchange (16) - : : : : +- BroadcastExchange (22) - : : : : +- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (31) - : : : +- * ColumnarToRow (30) - : : : +- CometFilter (29) - : : : +- CometScan parquet spark_catalog.default.store_sales (28) - : : +- BroadcastExchange (37) - : : +- * ColumnarToRow (36) - : : +- CometFilter (35) - : : +- CometScan parquet spark_catalog.default.customer_address (34) - : +- BroadcastExchange (43) - : +- * ColumnarToRow (42) - : +- CometFilter (41) - : +- CometScan parquet spark_catalog.default.store (40) - +- ReusedExchange (46) +TakeOrderedAndProject (56) ++- * HashAggregate (55) + +- Exchange (54) + +- * HashAggregate (53) + +- * HashAggregate (52) + +- Exchange (51) + +- * HashAggregate (50) + +- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (40) + : : +- * BroadcastHashJoin Inner BuildRight (39) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * HashAggregate (28) + : : : : +- Exchange (27) + : : : : +- * ColumnarToRow (26) + : : : : +- CometHashAggregate (25) + : : : : +- CometProject (24) + : : : : +- CometBroadcastHashJoin (23) + : : : : :- CometProject (19) + : : : : : +- CometBroadcastHashJoin (18) + : : : : : :- CometProject (13) + : : : : : : +- CometBroadcastHashJoin (12) + : : : : : : :- CometUnion (7) + : : : : : : : :- CometProject (3) + : : : : : : : : +- CometFilter (2) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : +- CometProject (6) + : : : : : : : +- CometFilter (5) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (4) + : : : : : : +- CometBroadcastExchange (11) + : : : : : : +- CometProject (10) + : : : : : : +- CometFilter (9) + : : : : : : +- CometScan parquet spark_catalog.default.item (8) + : : : : : +- CometBroadcastExchange (17) + : : : : : +- CometProject (16) + : : : : : +- CometFilter (15) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (14) + : : : : +- CometBroadcastExchange (22) + : : : : +- CometFilter (21) + : : : : +- CometScan parquet spark_catalog.default.customer (20) + : : : +- BroadcastExchange (32) + : : : +- * ColumnarToRow (31) + : : : +- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.store_sales (29) + : : +- BroadcastExchange (38) + : : +- * ColumnarToRow (37) + : : +- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.customer_address (35) + : +- BroadcastExchange (44) + : +- * ColumnarToRow (43) + : +- CometFilter (42) + : +- CometScan parquet spark_catalog.default.store (41) + +- ReusedExchange (47) (1) Scan parquet spark_catalog.default.catalog_sales @@ -92,388 +93,393 @@ Arguments: [sold_date_sk#11, customer_sk#12, item_sk#13], [ws_sold_date_sk#10 AS Child 0 Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] Child 1 Input [3]: [sold_date_sk#11, customer_sk#12, item_sk#13] -(8) ColumnarToRow [codegen id : 4] -Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] - -(9) Scan parquet spark_catalog.default.item +(8) Scan parquet spark_catalog.default.item Output [3]: [i_item_sk#14, i_class#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter +(9) CometFilter Input [3]: [i_item_sk#14, i_class#15, i_category#16] Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women )) AND (i_class#15 = maternity )) AND isnotnull(i_item_sk#14)) -(11) CometProject +(10) CometProject Input [3]: [i_item_sk#14, i_class#15, i_category#16] Arguments: [i_item_sk#14], [i_item_sk#14] -(12) ColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#14] - -(13) BroadcastExchange +(11) CometBroadcastExchange Input [1]: [i_item_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#14] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] +Right output [1]: [i_item_sk#14] +Arguments: [item_sk#7], [i_item_sk#14], Inner, BuildRight -(15) Project [codegen id : 4] -Output [2]: [sold_date_sk#5, customer_sk#6] +(13) CometProject Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] +Arguments: [sold_date_sk#5, customer_sk#6], [sold_date_sk#5, customer_sk#6] -(16) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#17] +(14) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#17, d_year#18, d_moy#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sold_date_sk#5] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(15) CometFilter +Input [3]: [d_date_sk#17, d_year#18, d_moy#19] +Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) -(18) Project [codegen id : 4] -Output [1]: [customer_sk#6] +(16) CometProject +Input [3]: [d_date_sk#17, d_year#18, d_moy#19] +Arguments: [d_date_sk#17], [d_date_sk#17] + +(17) CometBroadcastExchange +Input [1]: [d_date_sk#17] +Arguments: [d_date_sk#17] + +(18) CometBroadcastHashJoin +Left output [2]: [sold_date_sk#5, customer_sk#6] +Right output [1]: [d_date_sk#17] +Arguments: [sold_date_sk#5], [d_date_sk#17], Inner, BuildRight + +(19) CometProject Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] +Arguments: [customer_sk#6], [customer_sk#6] -(19) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] +(20) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#20, c_current_addr_sk#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(20) CometFilter -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) +(21) CometFilter +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] +Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_current_addr_sk#21)) -(21) ColumnarToRow [codegen id : 3] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +(22) CometBroadcastExchange +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] +Arguments: [c_customer_sk#20, c_current_addr_sk#21] -(22) BroadcastExchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(23) CometBroadcastHashJoin +Left output [1]: [customer_sk#6] +Right output [2]: [c_customer_sk#20, c_current_addr_sk#21] +Arguments: [customer_sk#6], [c_customer_sk#20], Inner, BuildRight -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_sk#6] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 4] -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] -Input [3]: [customer_sk#6, c_customer_sk#18, c_current_addr_sk#19] +(24) CometProject +Input [3]: [customer_sk#6, c_customer_sk#20, c_current_addr_sk#21] +Arguments: [c_customer_sk#20, c_current_addr_sk#21], [c_customer_sk#20, c_current_addr_sk#21] -(25) HashAggregate [codegen id : 4] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] +(25) CometHashAggregate +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] +Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] Functions: [] -Aggregate Attributes: [] -Results [2]: [c_customer_sk#18, c_current_addr_sk#19] -(26) Exchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: hashpartitioning(c_customer_sk#18, c_current_addr_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(26) ColumnarToRow [codegen id : 1] +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -(27) HashAggregate [codegen id : 9] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] +(27) Exchange +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] +Arguments: hashpartitioning(c_customer_sk#20, c_current_addr_sk#21, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(28) HashAggregate [codegen id : 6] +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] +Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] Functions: [] Aggregate Attributes: [] -Results [2]: [c_customer_sk#18, c_current_addr_sk#19] +Results [2]: [c_customer_sk#20, c_current_addr_sk#21] -(28) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +(29) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#25)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(29) CometFilter -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#20) +(30) CometFilter +Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] +Condition : isnotnull(ss_customer_sk#22) -(30) ColumnarToRow [codegen id : 5] -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +(31) ColumnarToRow [codegen id : 2] +Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -(31) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(32) BroadcastExchange +Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#18] -Right keys [1]: [ss_customer_sk#20] +(33) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#20] +Right keys [1]: [ss_customer_sk#22] Join type: Inner Join condition: None -(33) Project [codegen id : 9] -Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +(34) Project [codegen id : 6] +Output [4]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] +Input [5]: [c_customer_sk#20, c_current_addr_sk#21, ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -(34) Scan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#24, ca_county#25, ca_state#26] +(35) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#26, ca_county#27, ca_state#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county), IsNotNull(ca_state)] ReadSchema: struct -(35) CometFilter -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnotnull(ca_state#26)) +(36) CometFilter +Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] +Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_county#27)) AND isnotnull(ca_state#28)) -(36) ColumnarToRow [codegen id : 6] -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] +(37) ColumnarToRow [codegen id : 3] +Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -(37) BroadcastExchange -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(38) BroadcastExchange +Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#19] -Right keys [1]: [ca_address_sk#24] +(39) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_current_addr_sk#21] +Right keys [1]: [ca_address_sk#26] Join type: Inner Join condition: None -(39) Project [codegen id : 9] -Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26] -Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#26] +(40) Project [codegen id : 6] +Output [5]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28] +Input [7]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_address_sk#26, ca_county#27, ca_state#28] -(40) Scan parquet spark_catalog.default.store -Output [2]: [s_county#27, s_state#28] +(41) Scan parquet spark_catalog.default.store +Output [2]: [s_county#29, s_state#30] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] ReadSchema: struct -(41) CometFilter -Input [2]: [s_county#27, s_state#28] -Condition : (isnotnull(s_county#27) AND isnotnull(s_state#28)) +(42) CometFilter +Input [2]: [s_county#29, s_state#30] +Condition : (isnotnull(s_county#29) AND isnotnull(s_state#30)) -(42) ColumnarToRow [codegen id : 7] -Input [2]: [s_county#27, s_state#28] +(43) ColumnarToRow [codegen id : 4] +Input [2]: [s_county#29, s_state#30] -(43) BroadcastExchange -Input [2]: [s_county#27, s_state#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [plan_id=6] +(44) BroadcastExchange +Input [2]: [s_county#29, s_state#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [plan_id=4] -(44) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [ca_county#25, ca_state#26] -Right keys [2]: [s_county#27, s_state#28] +(45) BroadcastHashJoin [codegen id : 6] +Left keys [2]: [ca_county#27, ca_state#28] +Right keys [2]: [s_county#29, s_state#30] Join type: Inner Join condition: None -(45) Project [codegen id : 9] -Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26, s_county#27, s_state#28] +(46) Project [codegen id : 6] +Output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] +Input [7]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28, s_county#29, s_state#30] -(46) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#29] +(47) ReusedExchange [Reuses operator id: 66] +Output [1]: [d_date_sk#31] -(47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#29] +(48) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#24] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(48) Project [codegen id : 9] -Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#29] - -(49) HashAggregate [codegen id : 9] -Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Keys [1]: [c_customer_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum#30] -Results [2]: [c_customer_sk#18, sum#31] - -(50) Exchange -Input [2]: [c_customer_sk#18, sum#31] -Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(51) HashAggregate [codegen id : 10] -Input [2]: [c_customer_sk#18, sum#31] -Keys [1]: [c_customer_sk#18] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#32] -Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#32,17,2) / 50) as int) AS segment#33] - -(52) HashAggregate [codegen id : 10] -Input [1]: [segment#33] -Keys [1]: [segment#33] +(49) Project [codegen id : 6] +Output [2]: [c_customer_sk#20, ss_ext_sales_price#23] +Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#31] + +(50) HashAggregate [codegen id : 6] +Input [2]: [c_customer_sk#20, ss_ext_sales_price#23] +Keys [1]: [c_customer_sk#20] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#23))] +Aggregate Attributes [1]: [sum#32] +Results [2]: [c_customer_sk#20, sum#33] + +(51) Exchange +Input [2]: [c_customer_sk#20, sum#33] +Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(52) HashAggregate [codegen id : 7] +Input [2]: [c_customer_sk#20, sum#33] +Keys [1]: [c_customer_sk#20] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#23))#34] +Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#23))#34,17,2) / 50) as int) AS segment#35] + +(53) HashAggregate [codegen id : 7] +Input [1]: [segment#35] +Keys [1]: [segment#35] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [2]: [segment#33, count#35] +Aggregate Attributes [1]: [count#36] +Results [2]: [segment#35, count#37] -(53) Exchange -Input [2]: [segment#33, count#35] -Arguments: hashpartitioning(segment#33, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(54) Exchange +Input [2]: [segment#35, count#37] +Arguments: hashpartitioning(segment#35, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(54) HashAggregate [codegen id : 11] -Input [2]: [segment#33, count#35] -Keys [1]: [segment#33] +(55) HashAggregate [codegen id : 8] +Input [2]: [segment#35, count#37] +Keys [1]: [segment#35] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [3]: [segment#33, count(1)#36 AS num_customers#37, (segment#33 * 50) AS segment_base#38] +Aggregate Attributes [1]: [count(1)#38] +Results [3]: [segment#35, count(1)#38 AS num_customers#39, (segment#35 * 50) AS segment_base#40] -(55) TakeOrderedAndProject -Input [3]: [segment#33, num_customers#37, segment_base#38] -Arguments: 100, [segment#33 ASC NULLS FIRST, num_customers#37 ASC NULLS FIRST], [segment#33, num_customers#37, segment_base#38] +(56) TakeOrderedAndProject +Input [3]: [segment#35, num_customers#39, segment_base#40] +Arguments: 100, [segment#35 ASC NULLS FIRST, num_customers#39 ASC NULLS FIRST], [segment#35, num_customers#39, segment_base#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (60) -+- * ColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.date_dim (56) +BroadcastExchange (61) ++- * ColumnarToRow (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.date_dim (57) -(56) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#39, d_moy#40] +(57) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#17, d_year#18, d_moy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter -Input [3]: [d_date_sk#17, d_year#39, d_moy#40] -Condition : ((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 = 12)) AND (d_year#39 = 1998)) AND isnotnull(d_date_sk#17)) +(58) CometFilter +Input [3]: [d_date_sk#17, d_year#18, d_moy#19] +Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) -(58) CometProject -Input [3]: [d_date_sk#17, d_year#39, d_moy#40] +(59) CometProject +Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(59) ColumnarToRow [codegen id : 1] +(60) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] -(60) BroadcastExchange +(61) BroadcastExchange Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (65) -+- * ColumnarToRow (64) - +- CometProject (63) - +- CometFilter (62) - +- CometScan parquet spark_catalog.default.date_dim (61) +Subquery:3 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 +BroadcastExchange (66) ++- * ColumnarToRow (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) -(61) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#29, d_month_seq#41] +(62) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#31, d_month_seq#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#42), LessThanOrEqual(d_month_seq,ScalarSubquery#43), IsNotNull(d_date_sk)] ReadSchema: struct -(62) CometFilter -Input [2]: [d_date_sk#29, d_month_seq#41] -Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= ReusedSubquery Subquery scalar-subquery#42, [id=#44])) AND (d_month_seq#41 <= ReusedSubquery Subquery scalar-subquery#43, [id=#45])) AND isnotnull(d_date_sk#29)) +(63) CometFilter +Input [2]: [d_date_sk#31, d_month_seq#41] +Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= ReusedSubquery Subquery scalar-subquery#42, [id=#44])) AND (d_month_seq#41 <= ReusedSubquery Subquery scalar-subquery#43, [id=#45])) AND isnotnull(d_date_sk#31)) -(63) CometProject -Input [2]: [d_date_sk#29, d_month_seq#41] -Arguments: [d_date_sk#29], [d_date_sk#29] +(64) CometProject +Input [2]: [d_date_sk#31, d_month_seq#41] +Arguments: [d_date_sk#31], [d_date_sk#31] -(64) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#29] +(65) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#31] -(65) BroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +(66) BroadcastExchange +Input [1]: [d_date_sk#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:4 Hosting operator id = 62 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#44] +Subquery:4 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#44] -Subquery:5 Hosting operator id = 62 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#45] +Subquery:5 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#45] -Subquery:6 Hosting operator id = 61 Hosting Expression = Subquery scalar-subquery#42, [id=#44] -* HashAggregate (72) -+- Exchange (71) - +- * ColumnarToRow (70) - +- CometHashAggregate (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan parquet spark_catalog.default.date_dim (66) +Subquery:6 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#42, [id=#44] +* HashAggregate (73) ++- Exchange (72) + +- * ColumnarToRow (71) + +- CometHashAggregate (70) + +- CometProject (69) + +- CometFilter (68) + +- CometScan parquet spark_catalog.default.date_dim (67) -(66) Scan parquet spark_catalog.default.date_dim +(67) Scan parquet spark_catalog.default.date_dim Output [3]: [d_month_seq#46, d_year#47, d_moy#48] 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,12)] ReadSchema: struct -(67) CometFilter +(68) CometFilter Input [3]: [d_month_seq#46, d_year#47, d_moy#48] Condition : (((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 12)) -(68) CometProject +(69) CometProject Input [3]: [d_month_seq#46, d_year#47, d_moy#48] Arguments: [(d_month_seq + 1)#49], [(d_month_seq#46 + 1) AS (d_month_seq + 1)#49] -(69) CometHashAggregate +(70) CometHashAggregate Input [1]: [(d_month_seq + 1)#49] Keys [1]: [(d_month_seq + 1)#49] Functions: [] -(70) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 1)#49] -(71) Exchange +(72) Exchange Input [1]: [(d_month_seq + 1)#49] -Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(72) HashAggregate [codegen id : 2] +(73) HashAggregate [codegen id : 2] Input [1]: [(d_month_seq + 1)#49] Keys [1]: [(d_month_seq + 1)#49] Functions: [] Aggregate Attributes: [] Results [1]: [(d_month_seq + 1)#49] -Subquery:7 Hosting operator id = 61 Hosting Expression = Subquery scalar-subquery#43, [id=#45] -* HashAggregate (79) -+- Exchange (78) - +- * ColumnarToRow (77) - +- CometHashAggregate (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +Subquery:7 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#43, [id=#45] +* HashAggregate (80) ++- Exchange (79) + +- * ColumnarToRow (78) + +- CometHashAggregate (77) + +- CometProject (76) + +- CometFilter (75) + +- CometScan parquet spark_catalog.default.date_dim (74) -(73) Scan parquet spark_catalog.default.date_dim +(74) Scan parquet spark_catalog.default.date_dim Output [3]: [d_month_seq#50, d_year#51, d_moy#52] 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,12)] ReadSchema: struct -(74) CometFilter +(75) CometFilter Input [3]: [d_month_seq#50, d_year#51, d_moy#52] Condition : (((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 1998)) AND (d_moy#52 = 12)) -(75) CometProject +(76) CometProject Input [3]: [d_month_seq#50, d_year#51, d_moy#52] Arguments: [(d_month_seq + 3)#53], [(d_month_seq#50 + 3) AS (d_month_seq + 3)#53] -(76) CometHashAggregate +(77) CometHashAggregate Input [1]: [(d_month_seq + 3)#53] Keys [1]: [(d_month_seq + 3)#53] Functions: [] -(77) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 3)#53] -(78) Exchange +(79) Exchange Input [1]: [(d_month_seq + 3)#53] -Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(79) HashAggregate [codegen id : 2] +(80) HashAggregate [codegen id : 2] Input [1]: [(d_month_seq + 3)#53] Keys [1]: [(d_month_seq + 3)#53] Functions: [] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt index 30ba4b743b..32e9a92a7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (11) + WholeStageCodegen (8) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] InputAdapter Exchange [segment] #1 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [segment] [count,count] HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] InputAdapter Exchange [c_customer_sk] #2 - WholeStageCodegen (9) + WholeStageCodegen (6) HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] Project [c_customer_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -21,16 +21,16 @@ TakeOrderedAndProject [segment,num_customers,segment_base] HashAggregate [c_customer_sk,c_current_addr_sk] InputAdapter Exchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (4) - HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - BroadcastHashJoin [customer_sk,c_customer_sk] - Project [customer_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometProject [c_customer_sk,c_current_addr_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometProject [customer_sk] + CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometProject [sold_date_sk,customer_sk] + CometBroadcastHashJoin [item_sk,i_item_sk] CometUnion CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] CometFilter [cs_item_sk,cs_bill_customer_sk] @@ -47,32 +47,26 @@ TakeOrderedAndProject [segment,num_customers,segment_base] CometFilter [ws_item_sk,ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #5 + CometProject [i_item_sk] + CometFilter [i_category,i_class,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #7 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) + BroadcastExchange #8 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -85,7 +79,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] InputAdapter - Exchange [(d_month_seq + 1)] #9 + Exchange [(d_month_seq + 1)] #10 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -97,7 +91,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) HashAggregate [(d_month_seq + 3)] InputAdapter - Exchange [(d_month_seq + 3)] #10 + Exchange [(d_month_seq + 3)] #11 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -106,18 +100,18 @@ TakeOrderedAndProject [segment,num_customers,segment_base] CometFilter [d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (6) + BroadcastExchange #12 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [ca_address_sk,ca_county,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (7) + BroadcastExchange #13 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [s_county,s_state] CometScan parquet spark_catalog.default.store [s_county,s_state] InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [d_date_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt index 3d1d689bc9..dcd32a96a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt @@ -1,25 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (19) ++- * HashAggregate (18) + +- Exchange (17) + +- * HashAggregate (16) + +- * ColumnarToRow (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- CometBroadcastExchange (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.store_sales (4) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.item (9) (1) Scan parquet spark_catalog.default.date_dim @@ -37,10 +35,7 @@ Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) Input [3]: [d_date_sk#1, d_year#2, d_moy#3] Arguments: [d_date_sk#1], [d_date_sk#1] -(4) ColumnarToRow [codegen id : 3] -Input [1]: [d_date_sk#1] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -48,78 +43,73 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Condition : isnotnull(ss_item_sk#4) -(7) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [1]: [d_date_sk#1] +Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight -(10) Project [codegen id : 3] -Output [2]: [ss_item_sk#4, ss_ext_sales_price#5] +(8) CometProject Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [ss_item_sk#4, ss_ext_sales_price#5], [ss_item_sk#4, ss_ext_sales_price#5] -(11) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] 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 -(12) CometFilter +(10) CometFilter Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) -(13) CometProject +(11) CometProject Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9], [i_item_sk#7, i_brand_id#8, i_brand#9] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9] -(15) BroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(13) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#4, ss_ext_sales_price#5] +Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#4] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +(14) CometProject Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] + +(15) ColumnarToRow [codegen id : 1] +Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(18) HashAggregate [codegen id : 3] +(16) HashAggregate [codegen id : 1] Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] Keys [2]: [i_brand#9, i_brand_id#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#11] Results [3]: [i_brand#9, i_brand_id#8, sum#12] -(19) Exchange +(17) Exchange Input [3]: [i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [3]: [i_brand#9, i_brand_id#8, sum#12] Keys [2]: [i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] Results [3]: [i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(21) TakeOrderedAndProject +(19) TakeOrderedAndProject Input [3]: [brand_id#14, brand#15, ext_price#16] Arguments: 100, [ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [brand_id#14, brand#15, ext_price#16] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt index 7a0fe88633..a404b9b378 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt @@ -1,31 +1,23 @@ TakeOrderedAndProject [ext_price,brand_id,brand] - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt index bbed7eea67..bbfd9dcafa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt @@ -7,61 +7,61 @@ TakeOrderedAndProject (63) :- * HashAggregate (28) : +- Exchange (27) : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) + : +- * ColumnarToRow (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.customer_address (9) + : +- CometBroadcastExchange (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) : +- CometProject (19) : +- CometFilter (18) : +- CometScan parquet spark_catalog.default.item (17) :- * HashAggregate (43) : +- Exchange (42) : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * ColumnarToRow (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) + : +- * ColumnarToRow (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) +- * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * ColumnarToRow (46) - : : : +- CometFilter (45) - : : : +- CometScan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) + +- * ColumnarToRow (55) + +- CometProject (54) + +- CometBroadcastHashJoin (53) + :- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometProject (48) + : : +- CometBroadcastHashJoin (47) + : : :- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet spark_catalog.default.store_sales @@ -76,295 +76,292 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(6) Project [codegen id : 5] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +(7) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -(7) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] +(9) Scan parquet spark_catalog.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 -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] +(10) CometFilter +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)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] +(11) CometProject +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] +Arguments: [ca_address_sk#9], [ca_address_sk#9] -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [ca_address_sk#9] +Arguments: [ca_address_sk#9] -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +Right output [1]: [ca_address_sk#9] +Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight -(13) Project [codegen id : 5] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] +(14) CometProject +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] +Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] -(14) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_item_id#10] +(15) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#11, i_item_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#9, i_item_id#10] +(16) CometFilter +Input [2]: [i_item_sk#11, i_item_id#12] +Condition : isnotnull(i_item_sk#11) (17) Scan parquet spark_catalog.default.item -Output [2]: [i_item_id#11, i_color#12] +Output [2]: [i_item_id#13, i_color#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [blanched ,burnished ,slate ])] ReadSchema: struct (18) CometFilter -Input [2]: [i_item_id#11, i_color#12] -Condition : i_color#12 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#13, i_color#14] +Condition : i_color#14 IN (slate ,blanched ,burnished ) (19) CometProject -Input [2]: [i_item_id#11, i_color#12] -Arguments: [i_item_id#11], [i_item_id#11] - -(20) ColumnarToRow [codegen id : 3] -Input [1]: [i_item_id#11] - -(21) BroadcastExchange -Input [1]: [i_item_id#11] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_id#10] -Right keys [1]: [i_item_id#11] -Join type: LeftSemi -Join condition: None - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_item_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#3, i_item_id#10] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#10] - -(26) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#3, i_item_id#10] -Keys [1]: [i_item_id#10] +Input [2]: [i_item_id#13, i_color#14] +Arguments: [i_item_id#13], [i_item_id#13] + +(20) CometBroadcastExchange +Input [1]: [i_item_id#13] +Arguments: [i_item_id#13] + +(21) CometBroadcastHashJoin +Left output [2]: [i_item_sk#11, i_item_id#12] +Right output [1]: [i_item_id#13] +Arguments: [i_item_id#12], [i_item_id#13], LeftSemi, BuildRight + +(22) CometBroadcastExchange +Input [2]: [i_item_sk#11, i_item_id#12] +Arguments: [i_item_sk#11, i_item_id#12] + +(23) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] +Right output [2]: [i_item_sk#11, i_item_id#12] +Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight + +(24) CometProject +Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#12] +Arguments: [ss_ext_sales_price#3, i_item_id#12], [ss_ext_sales_price#3, i_item_id#12] + +(25) ColumnarToRow [codegen id : 1] +Input [2]: [ss_ext_sales_price#3, i_item_id#12] + +(26) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_sales_price#3, i_item_id#12] +Keys [1]: [i_item_id#12] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [i_item_id#10, sum#14] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#12, sum#16] (27) Exchange -Input [2]: [i_item_id#10, sum#14] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [2]: [i_item_id#12, sum#16] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#10, sum#14] -Keys [1]: [i_item_id#10] +(28) HashAggregate [codegen id : 2] +Input [2]: [i_item_id#12, sum#16] +Keys [1]: [i_item_id#12] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_item_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#17] +Results [2]: [i_item_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS total_sales#18] (29) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Output [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_sold_date_sk#22 IN dynamicpruning#23)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (30) CometFilter -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) - -(31) ColumnarToRow [codegen id : 11] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] - -(32) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#22] - -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] - -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#23] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#23] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] - -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#24, i_item_id#25] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#24] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#19, i_item_id#25] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] - -(41) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#19, i_item_id#25] -Keys [1]: [i_item_id#25] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_item_id#25, sum#27] +Input [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22] +Condition : (isnotnull(cs_bill_addr_sk#19) AND isnotnull(cs_item_sk#20)) + +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#24] + +(32) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight + +(33) CometProject +Input [5]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22, d_date_sk#24] +Arguments: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21], [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21] + +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#25] + +(35) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21] +Right output [1]: [ca_address_sk#25] +Arguments: [cs_bill_addr_sk#19], [ca_address_sk#25], Inner, BuildRight + +(36) CometProject +Input [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, ca_address_sk#25] +Arguments: [cs_item_sk#20, cs_ext_sales_price#21], [cs_item_sk#20, cs_ext_sales_price#21] + +(37) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#26, i_item_id#27] + +(38) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#20, cs_ext_sales_price#21] +Right output [2]: [i_item_sk#26, i_item_id#27] +Arguments: [cs_item_sk#20], [i_item_sk#26], Inner, BuildRight + +(39) CometProject +Input [4]: [cs_item_sk#20, cs_ext_sales_price#21, i_item_sk#26, i_item_id#27] +Arguments: [cs_ext_sales_price#21, i_item_id#27], [cs_ext_sales_price#21, i_item_id#27] + +(40) ColumnarToRow [codegen id : 3] +Input [2]: [cs_ext_sales_price#21, i_item_id#27] + +(41) HashAggregate [codegen id : 3] +Input [2]: [cs_ext_sales_price#21, i_item_id#27] +Keys [1]: [i_item_id#27] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#21))] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#27, sum#29] (42) Exchange -Input [2]: [i_item_id#25, sum#27] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [2]: [i_item_id#27, sum#29] +Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#25, sum#27] -Keys [1]: [i_item_id#25] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] -Results [2]: [i_item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] +(43) HashAggregate [codegen id : 4] +Input [2]: [i_item_id#27, sum#29] +Keys [1]: [i_item_id#27] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#21))#30] +Results [2]: [i_item_id#27, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#21))#30,17,2) AS total_sales#31] (44) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (45) CometFilter -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) - -(46) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] - -(47) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#35] - -(48) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#35] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 17] -Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] - -(50) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#36] - -(51) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#31] -Right keys [1]: [ca_address_sk#36] -Join type: Inner -Join condition: None - -(52) Project [codegen id : 17] -Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] - -(53) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#37, i_item_id#38] - -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#30] -Right keys [1]: [i_item_sk#37] -Join type: Inner -Join condition: None - -(55) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#32, i_item_id#38] -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] - -(56) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#32, i_item_id#38] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum#39] -Results [2]: [i_item_id#38, sum#40] +Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] +Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) + +(46) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#37] + +(47) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight + +(48) CometProject +Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#37] +Arguments: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34], [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] + +(49) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#38] + +(50) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] +Right output [1]: [ca_address_sk#38] +Arguments: [ws_bill_addr_sk#33], [ca_address_sk#38], Inner, BuildRight + +(51) CometProject +Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#38] +Arguments: [ws_item_sk#32, ws_ext_sales_price#34], [ws_item_sk#32, ws_ext_sales_price#34] + +(52) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#39, i_item_id#40] + +(53) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#32, ws_ext_sales_price#34] +Right output [2]: [i_item_sk#39, i_item_id#40] +Arguments: [ws_item_sk#32], [i_item_sk#39], Inner, BuildRight + +(54) CometProject +Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#39, i_item_id#40] +Arguments: [ws_ext_sales_price#34, i_item_id#40], [ws_ext_sales_price#34, i_item_id#40] + +(55) ColumnarToRow [codegen id : 5] +Input [2]: [ws_ext_sales_price#34, i_item_id#40] + +(56) HashAggregate [codegen id : 5] +Input [2]: [ws_ext_sales_price#34, i_item_id#40] +Keys [1]: [i_item_id#40] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] +Aggregate Attributes [1]: [sum#41] +Results [2]: [i_item_id#40, sum#42] (57) Exchange -Input [2]: [i_item_id#38, sum#40] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [2]: [i_item_id#40, sum#42] +Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(58) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#38, sum#40] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] -Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] +(58) HashAggregate [codegen id : 6] +Input [2]: [i_item_id#40, sum#42] +Keys [1]: [i_item_id#40] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#34))#43] +Results [2]: [i_item_id#40, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34))#43,17,2) AS total_sales#44] (59) Union -(60) HashAggregate [codegen id : 19] -Input [2]: [i_item_id#10, total_sales#16] -Keys [1]: [i_item_id#10] -Functions [1]: [partial_sum(total_sales#16)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [3]: [i_item_id#10, sum#45, isEmpty#46] +(60) HashAggregate [codegen id : 7] +Input [2]: [i_item_id#12, total_sales#18] +Keys [1]: [i_item_id#12] +Functions [1]: [partial_sum(total_sales#18)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [3]: [i_item_id#12, sum#47, isEmpty#48] (61) Exchange -Input [3]: [i_item_id#10, sum#45, isEmpty#46] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [3]: [i_item_id#12, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(62) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#10, sum#45, isEmpty#46] -Keys [1]: [i_item_id#10] -Functions [1]: [sum(total_sales#16)] -Aggregate Attributes [1]: [sum(total_sales#16)#47] -Results [2]: [i_item_id#10, sum(total_sales#16)#47 AS total_sales#48] +(62) HashAggregate [codegen id : 8] +Input [3]: [i_item_id#12, sum#47, isEmpty#48] +Keys [1]: [i_item_id#12] +Functions [1]: [sum(total_sales#18)] +Aggregate Attributes [1]: [sum(total_sales#18)#49] +Results [2]: [i_item_id#12, sum(total_sales#18)#49 AS total_sales#50] (63) TakeOrderedAndProject -Input [2]: [i_item_id#10, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#10, total_sales#48] +Input [2]: [i_item_id#12, total_sales#50] +Arguments: 100, [total_sales#50 ASC NULLS FIRST], [i_item_id#12, total_sales#50] ===== Subqueries ===== @@ -377,18 +374,18 @@ BroadcastExchange (68) (64) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#49, d_moy#50] +Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (65) CometFilter -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2001)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#6)) +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) (66) CometProject -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] (67) ColumnarToRow [codegen id : 1] @@ -396,10 +393,10 @@ Input [1]: [d_date_sk#6] (68) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#22 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt index f781ed1f7b..a5cb59e571 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (20) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (19) + WholeStageCodegen (7) HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #2 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,75 +31,59 @@ TakeOrderedAndProject [total_sales,i_item_id] CometProject [d_date_sk] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_item_id] - CometFilter [i_color] - CometScan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (12) + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange #6 + CometBroadcastHashJoin [i_item_id,i_item_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #7 + CometProject [i_item_id] + CometFilter [i_color] + CometScan parquet spark_catalog.default.item [i_item_id,i_color] + WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) + Exchange [i_item_id] #8 + WholeStageCodegen (3) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) + Exchange [i_item_id] #9 + WholeStageCodegen (5) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt index 8746c36d68..76b1adf1a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt @@ -14,22 +14,22 @@ TakeOrderedAndProject (45) : : +- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) + : : +- * ColumnarToRow (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.call_center (13) : +- BroadcastExchange (35) @@ -57,10 +57,7 @@ ReadSchema: struct Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] - -(4) Scan parquet spark_catalog.default.catalog_sales +(3) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -68,39 +65,46 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#7), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] Condition : (isnotnull(cs_item_sk#5) AND isnotnull(cs_call_center_sk#4)) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +Arguments: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Right output [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Arguments: [i_item_sk#1], [cs_item_sk#5], Inner, BuildRight -(9) Project [codegen id : 4] -Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] +(7) CometProject Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Arguments: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7], [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_year#10, d_moy#11] + +(11) CometBroadcastHashJoin +Left output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] +Right output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [cs_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(12) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] +(12) CometProject Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] +Arguments: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11], [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] (13) Scan parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#12, cc_name#13] @@ -113,24 +117,23 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#12, cc_name#13] Condition : (isnotnull(cc_call_center_sk#12) AND isnotnull(cc_name#13)) -(15) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [2]: [cc_call_center_sk#12, cc_name#13] +Arguments: [cc_call_center_sk#12, cc_name#13] -(16) BroadcastExchange -Input [2]: [cc_call_center_sk#12, cc_name#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] +Right output [2]: [cc_call_center_sk#12, cc_name#13] +Arguments: [cs_call_center_sk#4], [cc_call_center_sk#12], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#4] -Right keys [1]: [cc_call_center_sk#12] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] +(17) CometProject Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11, cc_call_center_sk#12, cc_name#13] +Arguments: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13], [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] + +(18) ColumnarToRow [codegen id : 1] +Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 1] Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#6))] @@ -139,9 +142,9 @@ Results [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] (20) Exchange Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 2] Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] @@ -150,9 +153,9 @@ Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDeci (22) Exchange Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(23) Sort [codegen id : 6] +(23) Sort [codegen id : 3] Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 @@ -160,7 +163,7 @@ Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] +(25) Filter [codegen id : 4] Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) @@ -168,18 +171,18 @@ Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] -(27) Filter [codegen id : 22] +(27) Filter [codegen id : 13] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(28) Project [codegen id : 22] +(28) Project [codegen id : 13] Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] (29) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(30) HashAggregate [codegen id : 12] +(30) HashAggregate [codegen id : 6] Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] @@ -188,9 +191,9 @@ Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDe (31) Exchange Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 13] +(32) Sort [codegen id : 7] Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 @@ -198,28 +201,28 @@ Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#2 Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(34) Project [codegen id : 14] +(34) Project [codegen id : 8] Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] (35) BroadcastExchange Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 22] +(36) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] +(37) Project [codegen id : 13] Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] (38) ReusedExchange [Reuses operator id: 31] Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(39) Sort [codegen id : 20] +(39) Sort [codegen id : 11] Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 @@ -227,21 +230,21 @@ Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#3 Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(41) Project [codegen id : 21] +(41) Project [codegen id : 12] Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] (42) BroadcastExchange Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 22] +(43) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] +(44) Project [codegen id : 13] Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] @@ -251,7 +254,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name# ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (49) +- * ColumnarToRow (48) +- CometFilter (47) @@ -274,6 +277,6 @@ Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt index 3bc01343ae..ee91816c92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,74 +8,67 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter Exchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,cs_item_sk] CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange #3 + CometFilter [cs_item_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) + BroadcastExchange #7 + WholeStageCodegen (8) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) + WholeStageCodegen (7) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) + Exchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (6) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] InputAdapter ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #9 + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) + WholeStageCodegen (11) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt index 35fedb6d37..2c82f91e93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt @@ -1,53 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Filter (16) - : : +- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (10) - : +- BroadcastExchange (30) - : +- * Filter (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometFilter (18) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- BroadcastExchange (46) - +- * Filter (45) - +- * HashAggregate (44) - +- Exchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * ColumnarToRow (35) - : : +- CometFilter (34) - : : +- CometScan parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (36) - +- ReusedExchange (39) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Filter (20) + : : +- * HashAggregate (19) + : : +- Exchange (18) + : : +- * HashAggregate (17) + : : +- * ColumnarToRow (16) + : : +- CometProject (15) + : : +- CometBroadcastHashJoin (14) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.item (3) + : : +- CometBroadcastExchange (13) + : : +- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- ReusedExchange (10) + : +- BroadcastExchange (34) + : +- * Filter (33) + : +- * HashAggregate (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- * ColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) + : : +- ReusedExchange (23) + : +- ReusedExchange (26) + +- BroadcastExchange (50) + +- * Filter (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * ColumnarToRow (45) + +- CometProject (44) + +- CometBroadcastHashJoin (43) + :- CometProject (41) + : +- CometBroadcastHashJoin (40) + : :- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (39) + +- ReusedExchange (42) (1) Scan parquet spark_catalog.default.store_sales @@ -62,325 +66,335 @@ ReadSchema: struct Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#5, i_item_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [i_item_sk#5, i_item_id#6] Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [2]: [i_item_sk#5, i_item_id#6] +Arguments: [i_item_sk#5, i_item_id#6] -(7) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Right output [2]: [i_item_sk#5, i_item_id#6] +Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) Project [codegen id : 4] -Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] +(7) CometProject Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6] +Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] -(10) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#7] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_date#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) + +(10) ReusedExchange [Reuses operator id: 59] +Output [1]: [d_date#9] + +(11) CometBroadcastHashJoin +Left output [2]: [d_date_sk#7, d_date#8] +Right output [1]: [d_date#9] +Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight -(12) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#2, i_item_id#6] +(12) CometProject +Input [2]: [d_date_sk#7, d_date#8] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(14) CometBroadcastHashJoin +Left output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#3], [d_date_sk#7], Inner, BuildRight + +(15) CometProject Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, d_date_sk#7] +Arguments: [ss_ext_sales_price#2, i_item_id#6], [ss_ext_sales_price#2, i_item_id#6] + +(16) ColumnarToRow [codegen id : 1] +Input [2]: [ss_ext_sales_price#2, i_item_id#6] -(13) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#2, i_item_id#6] Keys [1]: [i_item_id#6] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#8] -Results [2]: [i_item_id#6, sum#9] +Aggregate Attributes [1]: [sum#10] +Results [2]: [i_item_id#6, sum#11] -(14) Exchange -Input [2]: [i_item_id#6, sum#9] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(18) Exchange +Input [2]: [i_item_id#6, sum#11] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#6, sum#9] +(19) HashAggregate [codegen id : 6] +Input [2]: [i_item_id#6, sum#11] Keys [1]: [i_item_id#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#10] -Results [2]: [i_item_id#6 AS item_id#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#10,17,2) AS ss_item_rev#12] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] +Results [2]: [i_item_id#6 AS item_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS ss_item_rev#14] -(16) Filter [codegen id : 15] -Input [2]: [item_id#11, ss_item_rev#12] -Condition : isnotnull(ss_item_rev#12) +(20) Filter [codegen id : 6] +Input [2]: [item_id#13, ss_item_rev#14] +Condition : isnotnull(ss_item_rev#14) -(17) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +(21) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(18) CometFilter -Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] -Condition : isnotnull(cs_item_sk#13) - -(19) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] - -(20) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#17, i_item_id#18] - -(21) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#13] -Right keys [1]: [i_item_sk#17] +(22) CometFilter +Input [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +Condition : isnotnull(cs_item_sk#15) + +(23) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#19, i_item_id#20] + +(24) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +Right output [2]: [i_item_sk#19, i_item_id#20] +Arguments: [cs_item_sk#15], [i_item_sk#19], Inner, BuildRight + +(25) CometProject +Input [5]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_sk#19, i_item_id#20] +Arguments: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20], [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20] + +(26) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#21] + +(27) CometBroadcastHashJoin +Left output [3]: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20] +Right output [1]: [d_date_sk#21] +Arguments: [cs_sold_date_sk#17], [d_date_sk#21], Inner, BuildRight + +(28) CometProject +Input [4]: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20, d_date_sk#21] +Arguments: [cs_ext_sales_price#16, i_item_id#20], [cs_ext_sales_price#16, i_item_id#20] + +(29) ColumnarToRow [codegen id : 2] +Input [2]: [cs_ext_sales_price#16, i_item_id#20] + +(30) HashAggregate [codegen id : 2] +Input [2]: [cs_ext_sales_price#16, i_item_id#20] +Keys [1]: [i_item_id#20] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#16))] +Aggregate Attributes [1]: [sum#22] +Results [2]: [i_item_id#20, sum#23] + +(31) Exchange +Input [2]: [i_item_id#20, sum#23] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(32) HashAggregate [codegen id : 3] +Input [2]: [i_item_id#20, sum#23] +Keys [1]: [i_item_id#20] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#16))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#16))#24] +Results [2]: [i_item_id#20 AS item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#24,17,2) AS cs_item_rev#26] + +(33) Filter [codegen id : 3] +Input [2]: [item_id#25, cs_item_rev#26] +Condition : isnotnull(cs_item_rev#26) + +(34) BroadcastExchange +Input [2]: [item_id#25, cs_item_rev#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#25] Join type: Inner -Join condition: None +Join condition: ((((cast(ss_item_rev#14 as decimal(19,3)) >= (0.9 * cs_item_rev#26)) AND (cast(ss_item_rev#14 as decimal(20,3)) <= (1.1 * cs_item_rev#26))) AND (cast(cs_item_rev#26 as decimal(19,3)) >= (0.9 * ss_item_rev#14))) AND (cast(cs_item_rev#26 as decimal(20,3)) <= (1.1 * ss_item_rev#14))) -(22) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18] -Input [5]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_sk#17, i_item_id#18] +(36) Project [codegen id : 6] +Output [3]: [item_id#13, ss_item_rev#14, cs_item_rev#26] +Input [4]: [item_id#13, ss_item_rev#14, item_id#25, cs_item_rev#26] -(23) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#19] - -(24) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#15] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#14, i_item_id#18] -Input [4]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18, d_date_sk#19] - -(26) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#14, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#14))] -Aggregate Attributes [1]: [sum#20] -Results [2]: [i_item_id#18, sum#21] - -(27) Exchange -Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(28) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#18, sum#21] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#14))#22] -Results [2]: [i_item_id#18 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#14))#22,17,2) AS cs_item_rev#24] - -(29) Filter [codegen id : 9] -Input [2]: [item_id#23, cs_item_rev#24] -Condition : isnotnull(cs_item_rev#24) - -(30) BroadcastExchange -Input [2]: [item_id#23, cs_item_rev#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#11] -Right keys [1]: [item_id#23] -Join type: Inner -Join condition: ((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) - -(32) Project [codegen id : 15] -Output [3]: [item_id#11, ss_item_rev#12, cs_item_rev#24] -Input [4]: [item_id#11, ss_item_rev#12, item_id#23, cs_item_rev#24] - -(33) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] +(37) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#25) - -(35) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] - -(36) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#29, i_item_id#30] - -(37) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#25] -Right keys [1]: [i_item_sk#29] +(38) CometFilter +Input [3]: [ws_item_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : isnotnull(ws_item_sk#27) + +(39) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#31, i_item_id#32] + +(40) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Right output [2]: [i_item_sk#31, i_item_id#32] +Arguments: [ws_item_sk#27], [i_item_sk#31], Inner, BuildRight + +(41) CometProject +Input [5]: [ws_item_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, i_item_sk#31, i_item_id#32] +Arguments: [ws_ext_sales_price#28, ws_sold_date_sk#29, i_item_id#32], [ws_ext_sales_price#28, ws_sold_date_sk#29, i_item_id#32] + +(42) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#33] + +(43) CometBroadcastHashJoin +Left output [3]: [ws_ext_sales_price#28, ws_sold_date_sk#29, i_item_id#32] +Right output [1]: [d_date_sk#33] +Arguments: [ws_sold_date_sk#29], [d_date_sk#33], Inner, BuildRight + +(44) CometProject +Input [4]: [ws_ext_sales_price#28, ws_sold_date_sk#29, i_item_id#32, d_date_sk#33] +Arguments: [ws_ext_sales_price#28, i_item_id#32], [ws_ext_sales_price#28, i_item_id#32] + +(45) ColumnarToRow [codegen id : 4] +Input [2]: [ws_ext_sales_price#28, i_item_id#32] + +(46) HashAggregate [codegen id : 4] +Input [2]: [ws_ext_sales_price#28, i_item_id#32] +Keys [1]: [i_item_id#32] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#32, sum#35] + +(47) Exchange +Input [2]: [i_item_id#32, sum#35] +Arguments: hashpartitioning(i_item_id#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(48) HashAggregate [codegen id : 5] +Input [2]: [i_item_id#32, sum#35] +Keys [1]: [i_item_id#32] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#28))#36] +Results [2]: [i_item_id#32 AS item_id#37, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#28))#36,17,2) AS ws_item_rev#38] + +(49) Filter [codegen id : 5] +Input [2]: [item_id#37, ws_item_rev#38] +Condition : isnotnull(ws_item_rev#38) + +(50) BroadcastExchange +Input [2]: [item_id#37, ws_item_rev#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(51) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#37] Join type: Inner -Join condition: None - -(38) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] -Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#29, i_item_id#30] +Join condition: ((((((((cast(ss_item_rev#14 as decimal(19,3)) >= (0.9 * ws_item_rev#38)) AND (cast(ss_item_rev#14 as decimal(20,3)) <= (1.1 * ws_item_rev#38))) AND (cast(cs_item_rev#26 as decimal(19,3)) >= (0.9 * ws_item_rev#38))) AND (cast(cs_item_rev#26 as decimal(20,3)) <= (1.1 * ws_item_rev#38))) AND (cast(ws_item_rev#38 as decimal(19,3)) >= (0.9 * ss_item_rev#14))) AND (cast(ws_item_rev#38 as decimal(20,3)) <= (1.1 * ss_item_rev#14))) AND (cast(ws_item_rev#38 as decimal(19,3)) >= (0.9 * cs_item_rev#26))) AND (cast(ws_item_rev#38 as decimal(20,3)) <= (1.1 * cs_item_rev#26))) -(39) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#31] +(52) Project [codegen id : 6] +Output [8]: [item_id#13, ss_item_rev#14, (((ss_item_rev#14 / ((ss_item_rev#14 + cs_item_rev#26) + ws_item_rev#38)) / 3) * 100) AS ss_dev#39, cs_item_rev#26, (((cs_item_rev#26 / ((ss_item_rev#14 + cs_item_rev#26) + ws_item_rev#38)) / 3) * 100) AS cs_dev#40, ws_item_rev#38, (((ws_item_rev#38 / ((ss_item_rev#14 + cs_item_rev#26) + ws_item_rev#38)) / 3) * 100) AS ws_dev#41, (((ss_item_rev#14 + cs_item_rev#26) + ws_item_rev#38) / 3) AS average#42] +Input [5]: [item_id#13, ss_item_rev#14, cs_item_rev#26, item_id#37, ws_item_rev#38] -(40) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#26, i_item_id#30] -Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30, d_date_sk#31] - -(42) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#26, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#30, sum#33] - -(43) Exchange -Input [2]: [i_item_id#30, sum#33] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(44) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#30, sum#33] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#34] -Results [2]: [i_item_id#30 AS item_id#35, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#34,17,2) AS ws_item_rev#36] - -(45) Filter [codegen id : 14] -Input [2]: [item_id#35, ws_item_rev#36] -Condition : isnotnull(ws_item_rev#36) - -(46) BroadcastExchange -Input [2]: [item_id#35, ws_item_rev#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(47) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#11] -Right keys [1]: [item_id#35] -Join type: Inner -Join condition: ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * ws_item_rev#36)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) - -(48) Project [codegen id : 15] -Output [8]: [item_id#11, ss_item_rev#12, (((ss_item_rev#12 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ss_dev#37, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS cs_dev#38, ws_item_rev#36, (((ws_item_rev#36 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ws_dev#39, (((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36) / 3) AS average#40] -Input [5]: [item_id#11, ss_item_rev#12, cs_item_rev#24, item_id#35, ws_item_rev#36] - -(49) TakeOrderedAndProject -Input [8]: [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] -Arguments: 100, [item_id#11 ASC NULLS FIRST, ss_item_rev#12 ASC NULLS FIRST], [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] +(53) TakeOrderedAndProject +Input [8]: [item_id#13, ss_item_rev#14, ss_dev#39, cs_item_rev#26, cs_dev#40, ws_item_rev#38, ws_dev#41, average#42] +Arguments: 100, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#39, cs_item_rev#26, cs_dev#40, ws_item_rev#38, ws_dev#41, average#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (60) -+- * Project (59) - +- * BroadcastHashJoin LeftSemi BuildRight (58) - :- * ColumnarToRow (52) - : +- CometFilter (51) - : +- CometScan parquet spark_catalog.default.date_dim (50) - +- BroadcastExchange (57) - +- * ColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan parquet spark_catalog.default.date_dim (53) - - -(50) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_date#41] +BroadcastExchange (63) ++- * ColumnarToRow (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometFilter (55) + : +- CometScan parquet spark_catalog.default.date_dim (54) + +- CometBroadcastExchange (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) + + +(54) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter -Input [2]: [d_date_sk#7, d_date#41] +(55) CometFilter +Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(52) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#7, d_date#41] - -(53) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#42, d_week_seq#43] +(56) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#9, d_week_seq#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#44)] ReadSchema: struct -(54) CometFilter -Input [2]: [d_date#42, d_week_seq#43] +(57) CometFilter +Input [2]: [d_date#9, d_week_seq#43] Condition : (isnotnull(d_week_seq#43) AND (d_week_seq#43 = ReusedSubquery Subquery scalar-subquery#44, [id=#45])) -(55) CometProject -Input [2]: [d_date#42, d_week_seq#43] -Arguments: [d_date#42], [d_date#42] +(58) CometProject +Input [2]: [d_date#9, d_week_seq#43] +Arguments: [d_date#9], [d_date#9] -(56) ColumnarToRow [codegen id : 1] -Input [1]: [d_date#42] +(59) CometBroadcastExchange +Input [1]: [d_date#9] +Arguments: [d_date#9] -(57) BroadcastExchange -Input [1]: [d_date#42] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=7] +(60) CometBroadcastHashJoin +Left output [2]: [d_date_sk#7, d_date#8] +Right output [1]: [d_date#9] +Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight -(58) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_date#41] -Right keys [1]: [d_date#42] -Join type: LeftSemi -Join condition: None +(61) CometProject +Input [2]: [d_date_sk#7, d_date#8] +Arguments: [d_date_sk#7], [d_date_sk#7] -(59) Project [codegen id : 2] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#41] +(62) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] -(60) BroadcastExchange +(63) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:2 Hosting operator id = 57 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:3 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* ColumnarToRow (64) -+- CometProject (63) - +- CometFilter (62) - +- CometScan parquet spark_catalog.default.date_dim (61) +Subquery:3 Hosting operator id = 56 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* ColumnarToRow (67) ++- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) -(61) Scan parquet spark_catalog.default.date_dim +(64) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#46, d_week_seq#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(62) CometFilter +(65) CometFilter Input [2]: [d_date#46, d_week_seq#47] Condition : (isnotnull(d_date#46) AND (d_date#46 = 2000-01-03)) -(63) CometProject +(66) CometProject Input [2]: [d_date#46, d_week_seq#47] Arguments: [d_week_seq#47], [d_week_seq#47] -(64) ColumnarToRow [codegen id : 1] +(67) ColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#47] -Subquery:4 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt index c7ed479add..959d98ba91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (15) + WholeStageCodegen (6) Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] Project [item_id,ss_item_rev,cs_item_rev] @@ -8,91 +8,83 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (2) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date,d_date] CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date] - CometFilter [d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #3 + CometProject [d_date] + CometFilter [d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_date] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange #4 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date,d_date] + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (9) + BroadcastExchange #6 + WholeStageCodegen (3) Filter [cs_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] InputAdapter - Exchange [i_item_id] #6 - WholeStageCodegen (8) + Exchange [i_item_id] #7 + WholeStageCodegen (2) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #4 + ReusedExchange [d_date_sk] #5 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (14) + BroadcastExchange #8 + WholeStageCodegen (5) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (13) + Exchange [i_item_id] #9 + WholeStageCodegen (4) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #4 + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt index 023c54bdd3..afa9351c3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt @@ -1,48 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * HashAggregate (12) - : : : +- Exchange (11) - : : : +- * HashAggregate (10) - : : : +- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.date_dim (4) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.date_dim (19) - +- BroadcastExchange (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * HashAggregate (27) - : : +- ReusedExchange (26) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometFilter (29) - : +- CometScan parquet spark_catalog.default.store (28) - +- BroadcastExchange (38) - +- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +TakeOrderedAndProject (43) ++- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * HashAggregate (11) + : : : +- Exchange (10) + : : : +- * HashAggregate (9) + : : : +- * ColumnarToRow (8) + : : : +- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (15) + : : +- * ColumnarToRow (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.store (12) + : +- BroadcastExchange (22) + : +- * ColumnarToRow (21) + : +- CometProject (20) + : +- CometFilter (19) + : +- CometScan parquet spark_catalog.default.date_dim (18) + +- BroadcastExchange (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * HashAggregate (26) + : : +- ReusedExchange (25) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.store (27) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (1) Scan parquet spark_catalog.default.store_sales @@ -57,200 +56,196 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) ColumnarToRow [codegen id : 2] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.date_dim +(3) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) -(6) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#6] -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Right output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight -(9) Project [codegen id : 2] -Output [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] +(7) CometProject Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#6] +Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] + +(8) ColumnarToRow [codegen id : 1] +Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] -(10) HashAggregate [codegen id : 2] +(9) HashAggregate [codegen id : 1] Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] Aggregate Attributes [7]: [sum#7, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13] Results [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] -(11) Exchange +(10) Exchange Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(12) HashAggregate [codegen id : 10] +(11) HashAggregate [codegen id : 8] Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27] Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21,17,2) AS sun_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22,17,2) AS mon_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23,17,2) AS tue_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24,17,2) AS wed_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25,17,2) AS thu_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26,17,2) AS fri_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27,17,2) AS sat_sales#34] -(13) Scan parquet spark_catalog.default.store +(12) Scan parquet spark_catalog.default.store Output [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(14) CometFilter +(13) CometFilter Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] Condition : (isnotnull(s_store_sk#35) AND isnotnull(s_store_id#36)) -(15) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 2] Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] -(16) BroadcastExchange +(15) BroadcastExchange Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 10] +(16) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#35] Join type: Inner Join condition: None -(18) Project [codegen id : 10] +(17) Project [codegen id : 8] Output [10]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37] Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_sk#35, s_store_id#36, s_store_name#37] -(19) Scan parquet spark_catalog.default.date_dim +(18) Scan parquet spark_catalog.default.date_dim Output [2]: [d_month_seq#38, d_week_seq#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] ReadSchema: struct -(20) CometFilter +(19) CometFilter Input [2]: [d_month_seq#38, d_week_seq#39] Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= 1212)) AND (d_month_seq#38 <= 1223)) AND isnotnull(d_week_seq#39)) -(21) CometProject +(20) CometProject Input [2]: [d_month_seq#38, d_week_seq#39] Arguments: [d_week_seq#39], [d_week_seq#39] -(22) ColumnarToRow [codegen id : 4] +(21) ColumnarToRow [codegen id : 3] Input [1]: [d_week_seq#39] -(23) BroadcastExchange +(22) BroadcastExchange Input [1]: [d_week_seq#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 10] +(23) BroadcastHashJoin [codegen id : 8] Left keys [1]: [d_week_seq#5] Right keys [1]: [d_week_seq#39] Join type: Inner Join condition: None -(25) Project [codegen id : 10] +(24) Project [codegen id : 8] Output [10]: [s_store_name#37 AS s_store_name1#40, d_week_seq#5 AS d_week_seq1#41, s_store_id#36 AS s_store_id1#42, sun_sales#28 AS sun_sales1#43, mon_sales#29 AS mon_sales1#44, tue_sales#30 AS tue_sales1#45, wed_sales#31 AS wed_sales1#46, thu_sales#32 AS thu_sales1#47, fri_sales#33 AS fri_sales1#48, sat_sales#34 AS sat_sales1#49] Input [11]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37, d_week_seq#39] -(26) ReusedExchange [Reuses operator id: 11] +(25) ReusedExchange [Reuses operator id: 10] Output [9]: [d_week_seq#50, ss_store_sk#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58] -(27) HashAggregate [codegen id : 9] +(26) HashAggregate [codegen id : 7] Input [9]: [d_week_seq#50, ss_store_sk#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58] Keys [2]: [d_week_seq#50, ss_store_sk#51] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Tuesday ) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#60 END))] Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#60 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#60 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Tuesday ) THEN ss_sales_price#60 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#60 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#60 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#60 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#60 END))#27] Results [9]: [d_week_seq#50, ss_store_sk#51, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#60 END))#21,17,2) AS sun_sales#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#60 END))#22,17,2) AS mon_sales#62, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Tuesday ) THEN ss_sales_price#60 END))#23,17,2) AS tue_sales#63, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#60 END))#24,17,2) AS wed_sales#64, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#60 END))#25,17,2) AS thu_sales#65, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#60 END))#26,17,2) AS fri_sales#66, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#60 END))#27,17,2) AS sat_sales#67] -(28) Scan parquet spark_catalog.default.store +(27) Scan parquet spark_catalog.default.store Output [2]: [s_store_sk#68, s_store_id#69] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(29) CometFilter +(28) CometFilter Input [2]: [s_store_sk#68, s_store_id#69] Condition : (isnotnull(s_store_sk#68) AND isnotnull(s_store_id#69)) -(30) ColumnarToRow [codegen id : 7] +(29) ColumnarToRow [codegen id : 5] Input [2]: [s_store_sk#68, s_store_id#69] -(31) BroadcastExchange +(30) BroadcastExchange Input [2]: [s_store_sk#68, s_store_id#69] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(31) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#51] Right keys [1]: [s_store_sk#68] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(32) Project [codegen id : 7] Output [9]: [d_week_seq#50, sun_sales#61, mon_sales#62, tue_sales#63, wed_sales#64, thu_sales#65, fri_sales#66, sat_sales#67, s_store_id#69] Input [11]: [d_week_seq#50, ss_store_sk#51, sun_sales#61, mon_sales#62, tue_sales#63, wed_sales#64, thu_sales#65, fri_sales#66, sat_sales#67, s_store_sk#68, s_store_id#69] -(34) Scan parquet spark_catalog.default.date_dim +(33) Scan parquet spark_catalog.default.date_dim Output [2]: [d_month_seq#70, d_week_seq#71] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] ReadSchema: struct -(35) CometFilter +(34) CometFilter Input [2]: [d_month_seq#70, d_week_seq#71] Condition : (((isnotnull(d_month_seq#70) AND (d_month_seq#70 >= 1224)) AND (d_month_seq#70 <= 1235)) AND isnotnull(d_week_seq#71)) -(36) CometProject +(35) CometProject Input [2]: [d_month_seq#70, d_week_seq#71] Arguments: [d_week_seq#71], [d_week_seq#71] -(37) ColumnarToRow [codegen id : 8] +(36) ColumnarToRow [codegen id : 6] Input [1]: [d_week_seq#71] -(38) BroadcastExchange +(37) BroadcastExchange Input [1]: [d_week_seq#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 7] Left keys [1]: [d_week_seq#50] Right keys [1]: [d_week_seq#71] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(39) Project [codegen id : 7] Output [9]: [d_week_seq#50 AS d_week_seq2#72, s_store_id#69 AS s_store_id2#73, sun_sales#61 AS sun_sales2#74, mon_sales#62 AS mon_sales2#75, tue_sales#63 AS tue_sales2#76, wed_sales#64 AS wed_sales2#77, thu_sales#65 AS thu_sales2#78, fri_sales#66 AS fri_sales2#79, sat_sales#67 AS sat_sales2#80] Input [10]: [d_week_seq#50, sun_sales#61, mon_sales#62, tue_sales#63, wed_sales#64, thu_sales#65, fri_sales#66, sat_sales#67, s_store_id#69, d_week_seq#71] -(41) BroadcastExchange +(40) BroadcastExchange Input [9]: [d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=6] -(42) BroadcastHashJoin [codegen id : 10] +(41) BroadcastHashJoin [codegen id : 8] Left keys [2]: [s_store_id1#42, d_week_seq1#41] Right keys [2]: [s_store_id2#73, (d_week_seq2#72 - 52)] Join type: Inner Join condition: None -(43) Project [codegen id : 10] +(42) Project [codegen id : 8] Output [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1#43 / sun_sales2#74) AS (sun_sales1 / sun_sales2)#81, (mon_sales1#44 / mon_sales2#75) AS (mon_sales1 / mon_sales2)#82, (tue_sales1#45 / tue_sales2#76) AS (tue_sales1 / tue_sales2)#83, (wed_sales1#46 / wed_sales2#77) AS (wed_sales1 / wed_sales2)#84, (thu_sales1#47 / thu_sales2#78) AS (thu_sales1 / thu_sales2)#85, (fri_sales1#48 / fri_sales2#79) AS (fri_sales1 / fri_sales2)#86, (sat_sales1#49 / sat_sales2#80) AS (sat_sales1 / sat_sales2)#87] Input [19]: [s_store_name1#40, d_week_seq1#41, s_store_id1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] -(44) TakeOrderedAndProject +(43) TakeOrderedAndProject Input [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#81, (mon_sales1 / mon_sales2)#82, (tue_sales1 / tue_sales2)#83, (wed_sales1 / wed_sales2)#84, (thu_sales1 / thu_sales2)#85, (fri_sales1 / fri_sales2)#86, (sat_sales1 / sat_sales2)#87] Arguments: 100, [s_store_name1#40 ASC NULLS FIRST, s_store_id1#42 ASC NULLS FIRST, d_week_seq1#41 ASC NULLS FIRST], [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#81, (mon_sales1 / mon_sales2)#82, (tue_sales1 / tue_sales2)#83, (wed_sales1 / wed_sales2)#84, (thu_sales1 / thu_sales2)#85, (fri_sales1 / fri_sales2)#86, (sat_sales1 / sat_sales2)#87] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt index 9ad61e946e..55cb85ddbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - WholeStageCodegen (10) + WholeStageCodegen (8) Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] @@ -9,31 +9,27 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq,ss_store_sk] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange #2 + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 - WholeStageCodegen (3) + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_week_seq] @@ -41,7 +37,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] InputAdapter BroadcastExchange #5 - WholeStageCodegen (9) + WholeStageCodegen (7) Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] @@ -51,14 +47,14 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt index f8ae10ebef..46a4bdbace 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt @@ -1,43 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Filter (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer_address (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (29) - +- * Filter (28) - +- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * ColumnarToRow (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.item (22) +TakeOrderedAndProject (40) ++- * Filter (39) + +- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer_address (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.customer (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.store_sales (8) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.date_dim (13) + +- BroadcastExchange (33) + +- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- BroadcastExchange (30) + +- * Filter (29) + +- * HashAggregate (28) + +- Exchange (27) + +- * HashAggregate (26) + +- * ColumnarToRow (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.item (23) (1) Scan parquet spark_catalog.default.customer_address @@ -51,38 +52,31 @@ ReadSchema: struct Input [2]: [ca_address_sk#1, ca_state#2] Condition : isnotnull(ca_address_sk#1) -(3) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#1, ca_state#2] - -(4) Scan parquet spark_catalog.default.customer +(3) Scan parquet spark_catalog.default.customer Output [2]: [c_customer_sk#3, c_current_addr_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Condition : (isnotnull(c_current_addr_sk#4) AND isnotnull(c_customer_sk#3)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [2]: [c_customer_sk#3, c_current_addr_sk#4] +Arguments: [c_customer_sk#3, c_current_addr_sk#4] -(7) BroadcastExchange -Input [2]: [c_customer_sk#3, c_current_addr_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [2]: [ca_address_sk#1, ca_state#2] +Right output [2]: [c_customer_sk#3, c_current_addr_sk#4] +Arguments: [ca_address_sk#1], [c_current_addr_sk#4], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ca_address_sk#1] -Right keys [1]: [c_current_addr_sk#4] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 7] -Output [2]: [ca_state#2, c_customer_sk#3] +(7) CometProject Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4] +Arguments: [ca_state#2, c_customer_sk#3], [ca_state#2, c_customer_sk#3] -(10) Scan parquet spark_catalog.default.store_sales +(8) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -90,220 +84,238 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Condition : (isnotnull(ss_customer_sk#6) AND isnotnull(ss_item_sk#5)) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -(13) BroadcastExchange -Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [2]: [ca_state#2, c_customer_sk#3] +Right output [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 7] -Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] +(12) CometProject Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7], [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] -(16) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#9] +(13) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_month_seq#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#11), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(14) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#10] +Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date_sk#9)) + +(15) CometProject +Input [2]: [d_date_sk#9, d_month_seq#10] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(16) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] -(18) Project [codegen id : 7] -Output [2]: [ca_state#2, ss_item_sk#5] +(17) CometBroadcastHashJoin +Left output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight + +(18) CometProject Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ca_state#2, ss_item_sk#5], [ca_state#2, ss_item_sk#5] + +(19) ColumnarToRow [codegen id : 4] +Input [2]: [ca_state#2, ss_item_sk#5] -(19) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#10, i_current_price#11, i_category#12] +(20) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#13, i_current_price#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] ReadSchema: struct -(20) CometFilter -Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] -Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10)) +(21) CometFilter +Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] +Condition : ((isnotnull(i_current_price#14) AND isnotnull(i_category#15)) AND isnotnull(i_item_sk#13)) -(21) ColumnarToRow [codegen id : 6] -Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] +(22) ColumnarToRow [codegen id : 3] +Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] -(22) Scan parquet spark_catalog.default.item -Output [2]: [i_current_price#13, i_category#14] +(23) Scan parquet spark_catalog.default.item +Output [2]: [i_current_price#16, i_category#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct -(23) CometFilter -Input [2]: [i_current_price#13, i_category#14] -Condition : isnotnull(i_category#14) - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#13, i_category#14] - -(25) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#13, i_category#14] -Keys [1]: [i_category#14] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] -Aggregate Attributes [2]: [sum#15, count#16] -Results [3]: [i_category#14, sum#17, count#18] - -(26) Exchange -Input [3]: [i_category#14, sum#17, count#18] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 5] -Input [3]: [i_category#14, sum#17, count#18] -Keys [1]: [i_category#14] -Functions [1]: [avg(UnscaledValue(i_current_price#13))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#19] -Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#19 / 100.0) as decimal(11,6)) AS avg(i_current_price)#20, i_category#14] - -(28) Filter [codegen id : 5] -Input [2]: [avg(i_current_price)#20, i_category#14] -Condition : isnotnull(avg(i_current_price)#20) - -(29) BroadcastExchange -Input [2]: [avg(i_current_price)#20, i_category#14] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_category#12] -Right keys [1]: [i_category#14] +(24) CometFilter +Input [2]: [i_current_price#16, i_category#17] +Condition : isnotnull(i_category#17) + +(25) ColumnarToRow [codegen id : 1] +Input [2]: [i_current_price#16, i_category#17] + +(26) HashAggregate [codegen id : 1] +Input [2]: [i_current_price#16, i_category#17] +Keys [1]: [i_category#17] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#16))] +Aggregate Attributes [2]: [sum#18, count#19] +Results [3]: [i_category#17, sum#20, count#21] + +(27) Exchange +Input [3]: [i_category#17, sum#20, count#21] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(28) HashAggregate [codegen id : 2] +Input [3]: [i_category#17, sum#20, count#21] +Keys [1]: [i_category#17] +Functions [1]: [avg(UnscaledValue(i_current_price#16))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#16))#22] +Results [2]: [cast((avg(UnscaledValue(i_current_price#16))#22 / 100.0) as decimal(11,6)) AS avg(i_current_price)#23, i_category#17] + +(29) Filter [codegen id : 2] +Input [2]: [avg(i_current_price)#23, i_category#17] +Condition : isnotnull(avg(i_current_price)#23) + +(30) BroadcastExchange +Input [2]: [avg(i_current_price)#23, i_category#17] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=2] + +(31) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_category#15] +Right keys [1]: [i_category#17] Join type: Inner -Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)) +Join condition: (cast(i_current_price#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#23)) -(31) Project [codegen id : 6] -Output [1]: [i_item_sk#10] -Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#20, i_category#14] +(32) Project [codegen id : 3] +Output [1]: [i_item_sk#13] +Input [5]: [i_item_sk#13, i_current_price#14, i_category#15, avg(i_current_price)#23, i_category#17] -(32) BroadcastExchange -Input [1]: [i_item_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) BroadcastExchange +Input [1]: [i_item_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 7] +(34) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#10] +Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(34) Project [codegen id : 7] +(35) Project [codegen id : 4] Output [1]: [ca_state#2] -Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#10] +Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#13] -(35) HashAggregate [codegen id : 7] +(36) HashAggregate [codegen id : 4] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#21] -Results [2]: [ca_state#2, count#22] +Aggregate Attributes [1]: [count#24] +Results [2]: [ca_state#2, count#25] -(36) Exchange -Input [2]: [ca_state#2, count#22] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(37) Exchange +Input [2]: [ca_state#2, count#25] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(37) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#22] +(38) HashAggregate [codegen id : 5] +Input [2]: [ca_state#2, count#25] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [2]: [ca_state#2 AS state#24, count(1)#23 AS cnt#25] +Aggregate Attributes [1]: [count(1)#26] +Results [2]: [ca_state#2 AS state#27, count(1)#26 AS cnt#28] -(38) Filter [codegen id : 8] -Input [2]: [state#24, cnt#25] -Condition : (cnt#25 >= 10) +(39) Filter [codegen id : 5] +Input [2]: [state#27, cnt#28] +Condition : (cnt#28 >= 10) -(39) TakeOrderedAndProject -Input [2]: [state#24, cnt#25] -Arguments: 100, [cnt#25 ASC NULLS FIRST], [state#24, cnt#25] +(40) TakeOrderedAndProject +Input [2]: [state#27, cnt#28] +Arguments: 100, [cnt#28 ASC NULLS FIRST], [state#27, cnt#28] ===== Subqueries ===== -Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +Subquery:1 Hosting operator id = 8 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) -(40) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#26] +(41) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_month_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#27), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#11), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#26] -Condition : ((isnotnull(d_month_seq#26) AND (d_month_seq#26 = ReusedSubquery Subquery scalar-subquery#27, [id=#28])) AND isnotnull(d_date_sk#9)) +(42) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#10] +Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date_sk#9)) -(42) CometProject -Input [2]: [d_date_sk#9, d_month_seq#26] +(43) CometProject +Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(43) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(44) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 41 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] +Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -Subquery:3 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* HashAggregate (51) -+- Exchange (50) - +- * ColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* HashAggregate (52) ++- Exchange (51) + +- * ColumnarToRow (50) + +- CometHashAggregate (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(45) Scan parquet spark_catalog.default.date_dim +(46) Scan parquet spark_catalog.default.date_dim Output [3]: [d_month_seq#29, d_year#30, d_moy#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(46) CometFilter +(47) CometFilter Input [3]: [d_month_seq#29, d_year#30, d_moy#31] Condition : (((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 1)) -(47) CometProject +(48) CometProject Input [3]: [d_month_seq#29, d_year#30, d_moy#31] Arguments: [d_month_seq#29], [d_month_seq#29] -(48) CometHashAggregate +(49) CometHashAggregate Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -(49) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_month_seq#29] -(50) Exchange +(51) Exchange Input [1]: [d_month_seq#29] -Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(51) HashAggregate [codegen id : 2] +(52) HashAggregate [codegen id : 2] Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] Aggregate Attributes: [] Results [1]: [d_month_seq#29] +Subquery:4 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] + +Subquery:5 Hosting operator id = 13 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt index c2d5a6ce83..a7c65f50ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt @@ -1,63 +1,59 @@ TakeOrderedAndProject [cnt,state] - WholeStageCodegen (8) + WholeStageCodegen (5) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] InputAdapter Exchange [ca_state] #1 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - ReusedSubquery [d_month_seq] #2 - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (2) - HashAggregate [d_month_seq] - InputAdapter - Exchange [d_month_seq] #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #2 + CometFilter [c_current_addr_sk,c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #3 + CometFilter [ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (2) + HashAggregate [d_month_seq] + InputAdapter + Exchange [d_month_seq] #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (3) Project [i_item_sk] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow @@ -65,13 +61,13 @@ TakeOrderedAndProject [cnt,state] CometFilter [i_current_price,i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) + BroadcastExchange #8 + WholeStageCodegen (2) Filter [avg(i_current_price)] HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter - Exchange [i_category] #8 - WholeStageCodegen (4) + Exchange [i_category] #9 + WholeStageCodegen (1) HashAggregate [i_category,i_current_price] [sum,count,sum,count] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt index 78f4b27ac9..342a3deb87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt @@ -7,61 +7,61 @@ TakeOrderedAndProject (63) :- * HashAggregate (28) : +- Exchange (27) : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) + : +- * ColumnarToRow (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.customer_address (9) + : +- CometBroadcastExchange (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) : +- CometProject (19) : +- CometFilter (18) : +- CometScan parquet spark_catalog.default.item (17) :- * HashAggregate (43) : +- Exchange (42) : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * ColumnarToRow (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) + : +- * ColumnarToRow (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) +- * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * ColumnarToRow (46) - : : : +- CometFilter (45) - : : : +- CometScan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) + +- * ColumnarToRow (55) + +- CometProject (54) + +- CometBroadcastHashJoin (53) + :- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometProject (48) + : : +- CometBroadcastHashJoin (47) + : : :- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet spark_catalog.default.store_sales @@ -76,295 +76,292 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(6) Project [codegen id : 5] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +(7) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -(7) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] +(9) Scan parquet spark_catalog.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 -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] +(10) CometFilter +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)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] +(11) CometProject +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] +Arguments: [ca_address_sk#9], [ca_address_sk#9] -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [ca_address_sk#9] +Arguments: [ca_address_sk#9] -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +Right output [1]: [ca_address_sk#9] +Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight -(13) Project [codegen id : 5] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] +(14) CometProject +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] +Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] -(14) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_item_id#10] +(15) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#11, i_item_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#9, i_item_id#10] +(16) CometFilter +Input [2]: [i_item_sk#11, i_item_id#12] +Condition : isnotnull(i_item_sk#11) (17) Scan parquet spark_catalog.default.item -Output [2]: [i_item_id#11, i_category#12] +Output [2]: [i_item_id#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (18) CometFilter -Input [2]: [i_item_id#11, i_category#12] -Condition : (isnotnull(i_category#12) AND (i_category#12 = Music )) +Input [2]: [i_item_id#13, i_category#14] +Condition : (isnotnull(i_category#14) AND (i_category#14 = Music )) (19) CometProject -Input [2]: [i_item_id#11, i_category#12] -Arguments: [i_item_id#11], [i_item_id#11] - -(20) ColumnarToRow [codegen id : 3] -Input [1]: [i_item_id#11] - -(21) BroadcastExchange -Input [1]: [i_item_id#11] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_id#10] -Right keys [1]: [i_item_id#11] -Join type: LeftSemi -Join condition: None - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_item_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#3, i_item_id#10] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#10] - -(26) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#3, i_item_id#10] -Keys [1]: [i_item_id#10] +Input [2]: [i_item_id#13, i_category#14] +Arguments: [i_item_id#13], [i_item_id#13] + +(20) CometBroadcastExchange +Input [1]: [i_item_id#13] +Arguments: [i_item_id#13] + +(21) CometBroadcastHashJoin +Left output [2]: [i_item_sk#11, i_item_id#12] +Right output [1]: [i_item_id#13] +Arguments: [i_item_id#12], [i_item_id#13], LeftSemi, BuildRight + +(22) CometBroadcastExchange +Input [2]: [i_item_sk#11, i_item_id#12] +Arguments: [i_item_sk#11, i_item_id#12] + +(23) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] +Right output [2]: [i_item_sk#11, i_item_id#12] +Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight + +(24) CometProject +Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#12] +Arguments: [ss_ext_sales_price#3, i_item_id#12], [ss_ext_sales_price#3, i_item_id#12] + +(25) ColumnarToRow [codegen id : 1] +Input [2]: [ss_ext_sales_price#3, i_item_id#12] + +(26) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_sales_price#3, i_item_id#12] +Keys [1]: [i_item_id#12] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [i_item_id#10, sum#14] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#12, sum#16] (27) Exchange -Input [2]: [i_item_id#10, sum#14] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [2]: [i_item_id#12, sum#16] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#10, sum#14] -Keys [1]: [i_item_id#10] +(28) HashAggregate [codegen id : 2] +Input [2]: [i_item_id#12, sum#16] +Keys [1]: [i_item_id#12] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_item_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#17] +Results [2]: [i_item_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS total_sales#18] (29) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Output [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_sold_date_sk#22 IN dynamicpruning#23)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (30) CometFilter -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) - -(31) ColumnarToRow [codegen id : 11] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] - -(32) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#22] - -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] - -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#23] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#23] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] - -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#24, i_item_id#25] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#24] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#19, i_item_id#25] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] - -(41) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#19, i_item_id#25] -Keys [1]: [i_item_id#25] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_item_id#25, sum#27] +Input [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22] +Condition : (isnotnull(cs_bill_addr_sk#19) AND isnotnull(cs_item_sk#20)) + +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#24] + +(32) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight + +(33) CometProject +Input [5]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, cs_sold_date_sk#22, d_date_sk#24] +Arguments: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21], [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21] + +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#25] + +(35) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21] +Right output [1]: [ca_address_sk#25] +Arguments: [cs_bill_addr_sk#19], [ca_address_sk#25], Inner, BuildRight + +(36) CometProject +Input [4]: [cs_bill_addr_sk#19, cs_item_sk#20, cs_ext_sales_price#21, ca_address_sk#25] +Arguments: [cs_item_sk#20, cs_ext_sales_price#21], [cs_item_sk#20, cs_ext_sales_price#21] + +(37) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#26, i_item_id#27] + +(38) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#20, cs_ext_sales_price#21] +Right output [2]: [i_item_sk#26, i_item_id#27] +Arguments: [cs_item_sk#20], [i_item_sk#26], Inner, BuildRight + +(39) CometProject +Input [4]: [cs_item_sk#20, cs_ext_sales_price#21, i_item_sk#26, i_item_id#27] +Arguments: [cs_ext_sales_price#21, i_item_id#27], [cs_ext_sales_price#21, i_item_id#27] + +(40) ColumnarToRow [codegen id : 3] +Input [2]: [cs_ext_sales_price#21, i_item_id#27] + +(41) HashAggregate [codegen id : 3] +Input [2]: [cs_ext_sales_price#21, i_item_id#27] +Keys [1]: [i_item_id#27] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#21))] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#27, sum#29] (42) Exchange -Input [2]: [i_item_id#25, sum#27] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [2]: [i_item_id#27, sum#29] +Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#25, sum#27] -Keys [1]: [i_item_id#25] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] -Results [2]: [i_item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] +(43) HashAggregate [codegen id : 4] +Input [2]: [i_item_id#27, sum#29] +Keys [1]: [i_item_id#27] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#21))#30] +Results [2]: [i_item_id#27, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#21))#30,17,2) AS total_sales#31] (44) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (45) CometFilter -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) - -(46) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] - -(47) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#35] - -(48) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#35] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 17] -Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] - -(50) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#36] - -(51) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#31] -Right keys [1]: [ca_address_sk#36] -Join type: Inner -Join condition: None - -(52) Project [codegen id : 17] -Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] - -(53) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#37, i_item_id#38] - -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#30] -Right keys [1]: [i_item_sk#37] -Join type: Inner -Join condition: None - -(55) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#32, i_item_id#38] -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] - -(56) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#32, i_item_id#38] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum#39] -Results [2]: [i_item_id#38, sum#40] +Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] +Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) + +(46) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#37] + +(47) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight + +(48) CometProject +Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#37] +Arguments: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34], [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] + +(49) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#38] + +(50) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] +Right output [1]: [ca_address_sk#38] +Arguments: [ws_bill_addr_sk#33], [ca_address_sk#38], Inner, BuildRight + +(51) CometProject +Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#38] +Arguments: [ws_item_sk#32, ws_ext_sales_price#34], [ws_item_sk#32, ws_ext_sales_price#34] + +(52) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#39, i_item_id#40] + +(53) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#32, ws_ext_sales_price#34] +Right output [2]: [i_item_sk#39, i_item_id#40] +Arguments: [ws_item_sk#32], [i_item_sk#39], Inner, BuildRight + +(54) CometProject +Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#39, i_item_id#40] +Arguments: [ws_ext_sales_price#34, i_item_id#40], [ws_ext_sales_price#34, i_item_id#40] + +(55) ColumnarToRow [codegen id : 5] +Input [2]: [ws_ext_sales_price#34, i_item_id#40] + +(56) HashAggregate [codegen id : 5] +Input [2]: [ws_ext_sales_price#34, i_item_id#40] +Keys [1]: [i_item_id#40] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] +Aggregate Attributes [1]: [sum#41] +Results [2]: [i_item_id#40, sum#42] (57) Exchange -Input [2]: [i_item_id#38, sum#40] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [2]: [i_item_id#40, sum#42] +Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(58) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#38, sum#40] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] -Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] +(58) HashAggregate [codegen id : 6] +Input [2]: [i_item_id#40, sum#42] +Keys [1]: [i_item_id#40] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#34))#43] +Results [2]: [i_item_id#40, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34))#43,17,2) AS total_sales#44] (59) Union -(60) HashAggregate [codegen id : 19] -Input [2]: [i_item_id#10, total_sales#16] -Keys [1]: [i_item_id#10] -Functions [1]: [partial_sum(total_sales#16)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [3]: [i_item_id#10, sum#45, isEmpty#46] +(60) HashAggregate [codegen id : 7] +Input [2]: [i_item_id#12, total_sales#18] +Keys [1]: [i_item_id#12] +Functions [1]: [partial_sum(total_sales#18)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [3]: [i_item_id#12, sum#47, isEmpty#48] (61) Exchange -Input [3]: [i_item_id#10, sum#45, isEmpty#46] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [3]: [i_item_id#12, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(62) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#10, sum#45, isEmpty#46] -Keys [1]: [i_item_id#10] -Functions [1]: [sum(total_sales#16)] -Aggregate Attributes [1]: [sum(total_sales#16)#47] -Results [2]: [i_item_id#10, sum(total_sales#16)#47 AS total_sales#48] +(62) HashAggregate [codegen id : 8] +Input [3]: [i_item_id#12, sum#47, isEmpty#48] +Keys [1]: [i_item_id#12] +Functions [1]: [sum(total_sales#18)] +Aggregate Attributes [1]: [sum(total_sales#18)#49] +Results [2]: [i_item_id#12, sum(total_sales#18)#49 AS total_sales#50] (63) TakeOrderedAndProject -Input [2]: [i_item_id#10, total_sales#48] -Arguments: 100, [i_item_id#10 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#10, total_sales#48] +Input [2]: [i_item_id#12, total_sales#50] +Arguments: 100, [i_item_id#12 ASC NULLS FIRST, total_sales#50 ASC NULLS FIRST], [i_item_id#12, total_sales#50] ===== Subqueries ===== @@ -377,18 +374,18 @@ BroadcastExchange (68) (64) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#49, d_moy#50] +Output [3]: [d_date_sk#6, d_year#7, d_moy#8] 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,9), IsNotNull(d_date_sk)] ReadSchema: struct (65) CometFilter -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 9)) AND isnotnull(d_date_sk#6)) +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) (66) CometProject -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] (67) ColumnarToRow [codegen id : 1] @@ -396,10 +393,10 @@ Input [1]: [d_date_sk#6] (68) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#22 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt index b010414a86..cf226bb147 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (20) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (19) + WholeStageCodegen (7) HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #2 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,75 +31,59 @@ TakeOrderedAndProject [i_item_id,total_sales] CometProject [d_date_sk] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_item_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (12) + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange #6 + CometBroadcastHashJoin [i_item_id,i_item_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #7 + CometProject [i_item_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_item_id,i_category] + WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) + Exchange [i_item_id] #8 + WholeStageCodegen (3) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) + Exchange [i_item_id] #9 + WholeStageCodegen (5) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt index 766362167e..6d0d03e136 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt @@ -1,71 +1,69 @@ == Physical Plan == -* Project (67) -+- * BroadcastNestedLoopJoin Inner BuildRight (66) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (26) - : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : :- * Project (20) - : : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * ColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan parquet spark_catalog.default.promotion (11) - : : : : +- ReusedExchange (18) - : : : +- BroadcastExchange (24) - : : : +- * ColumnarToRow (23) - : : : +- CometFilter (22) - : : : +- CometScan parquet spark_catalog.default.customer (21) - : : +- BroadcastExchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.customer_address (27) - : +- BroadcastExchange (38) - : +- * ColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan parquet spark_catalog.default.item (34) - +- BroadcastExchange (65) - +- * HashAggregate (64) - +- Exchange (63) - +- * HashAggregate (62) - +- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * Project (55) - : : +- * BroadcastHashJoin Inner BuildRight (54) - : : :- * Project (52) - : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : :- * Project (49) - : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : :- * ColumnarToRow (46) - : : : : : +- CometFilter (45) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (44) - : : : : +- ReusedExchange (47) - : : : +- ReusedExchange (50) - : : +- ReusedExchange (53) - : +- ReusedExchange (56) - +- ReusedExchange (59) +* Project (65) ++- * BroadcastNestedLoopJoin Inner BuildRight (64) + :- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * ColumnarToRow (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometProject (20) + : : : : +- CometBroadcastHashJoin (19) + : : : : :- CometProject (14) + : : : : : +- CometBroadcastHashJoin (13) + : : : : : :- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (6) + : : : : : : +- CometProject (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.store (3) + : : : : : +- CometBroadcastExchange (12) + : : : : : +- CometProject (11) + : : : : : +- CometFilter (10) + : : : : : +- CometScan parquet spark_catalog.default.promotion (9) + : : : : +- CometBroadcastExchange (18) + : : : : +- CometProject (17) + : : : : +- CometFilter (16) + : : : : +- CometScan parquet spark_catalog.default.date_dim (15) + : : : +- CometBroadcastExchange (23) + : : : +- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.customer (21) + : : +- CometBroadcastExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.customer_address (26) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometFilter (33) + : +- CometScan parquet spark_catalog.default.item (32) + +- BroadcastExchange (63) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * ColumnarToRow (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometProject (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometProject (49) + : : : +- CometBroadcastHashJoin (48) + : : : :- CometProject (46) + : : : : +- CometBroadcastHashJoin (45) + : : : : :- CometFilter (43) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (42) + : : : : +- ReusedExchange (44) + : : : +- ReusedExchange (47) + : : +- ReusedExchange (50) + : +- ReusedExchange (53) + +- ReusedExchange (56) (1) Scan parquet spark_catalog.default.store_sales @@ -80,338 +78,328 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [s_store_sk#8, s_gmt_offset#9] Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) -(6) CometProject +(5) CometProject Input [2]: [s_store_sk#8, s_gmt_offset#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(7) ColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#8] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [s_store_sk#8] -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Right output [1]: [s_store_sk#8] +Arguments: [ss_store_sk#3], [s_store_sk#8], Inner, BuildRight -(10) Project [codegen id : 7] -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +(8) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(11) Scan parquet spark_catalog.default.promotion +(9) Scan parquet spark_catalog.default.promotion Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [Or(Or(EqualTo(p_channel_dmail,Y),EqualTo(p_channel_email,Y)),EqualTo(p_channel_tv,Y)), IsNotNull(p_promo_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] Condition : ((((p_channel_dmail#11 = Y) OR (p_channel_email#12 = Y)) OR (p_channel_tv#13 = Y)) AND isnotnull(p_promo_sk#10)) -(13) CometProject +(11) CometProject Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] Arguments: [p_promo_sk#10], [p_promo_sk#10] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [1]: [p_promo_sk#10] +Arguments: [p_promo_sk#10] -(15) BroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(13) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Right output [1]: [p_promo_sk#10] +Arguments: [ss_promo_sk#4], [p_promo_sk#10], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_promo_sk#4] -Right keys [1]: [p_promo_sk#10] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] +(14) CometProject Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -(18) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#14] +(15) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#14, d_year#15, d_moy#16] +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 -(19) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(16) CometFilter +Input [3]: [d_date_sk#14, d_year#15, d_moy#16] +Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) + +(17) CometProject +Input [3]: [d_date_sk#14, d_year#15, d_moy#16] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(18) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(19) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#6], [d_date_sk#14], Inner, BuildRight -(20) Project [codegen id : 7] -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] +(20) CometProject Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] (21) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#15, c_current_addr_sk#16] +Output [2]: [c_customer_sk#17, c_current_addr_sk#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (22) CometFilter -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) +Input [2]: [c_customer_sk#17, c_current_addr_sk#18] +Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) -(23) ColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] +(23) CometBroadcastExchange +Input [2]: [c_customer_sk#17, c_current_addr_sk#18] +Arguments: [c_customer_sk#17, c_current_addr_sk#18] -(24) BroadcastExchange -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(24) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] +Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] +Arguments: [ss_customer_sk#2], [c_customer_sk#17], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 7] -Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#15, c_current_addr_sk#16] +(25) CometProject +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#17, c_current_addr_sk#18] +Arguments: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18], [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] -(27) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_gmt_offset#18] +(26) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#19, ca_gmt_offset#20] 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 -(28) CometFilter -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -5.00)) AND isnotnull(ca_address_sk#17)) +(27) CometFilter +Input [2]: [ca_address_sk#19, ca_gmt_offset#20] +Condition : ((isnotnull(ca_gmt_offset#20) AND (ca_gmt_offset#20 = -5.00)) AND isnotnull(ca_address_sk#19)) -(29) CometProject -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Arguments: [ca_address_sk#17], [ca_address_sk#17] +(28) CometProject +Input [2]: [ca_address_sk#19, ca_gmt_offset#20] +Arguments: [ca_address_sk#19], [ca_address_sk#19] -(30) ColumnarToRow [codegen id : 5] -Input [1]: [ca_address_sk#17] +(29) CometBroadcastExchange +Input [1]: [ca_address_sk#19] +Arguments: [ca_address_sk#19] -(31) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(30) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] +Right output [1]: [ca_address_sk#19] +Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#16] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None +(31) CometProject +Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18, ca_address_sk#19] +Arguments: [ss_item_sk#1, ss_ext_sales_price#5], [ss_item_sk#1, ss_ext_sales_price#5] -(33) Project [codegen id : 7] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16, ca_address_sk#17] - -(34) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_category#20] +(32) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#21, i_category#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Jewelry ), IsNotNull(i_item_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [i_item_sk#19, i_category#20] -Condition : ((isnotnull(i_category#20) AND (i_category#20 = Jewelry )) AND isnotnull(i_item_sk#19)) +(33) CometFilter +Input [2]: [i_item_sk#21, i_category#22] +Condition : ((isnotnull(i_category#22) AND (i_category#22 = Jewelry )) AND isnotnull(i_item_sk#21)) -(36) CometProject -Input [2]: [i_item_sk#19, i_category#20] -Arguments: [i_item_sk#19], [i_item_sk#19] +(34) CometProject +Input [2]: [i_item_sk#21, i_category#22] +Arguments: [i_item_sk#21], [i_item_sk#21] -(37) ColumnarToRow [codegen id : 6] -Input [1]: [i_item_sk#19] +(35) CometBroadcastExchange +Input [1]: [i_item_sk#21] +Arguments: [i_item_sk#21] -(38) BroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#1, ss_ext_sales_price#5] +Right output [1]: [i_item_sk#21] +Arguments: [ss_item_sk#1], [i_item_sk#21], Inner, BuildRight -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None +(37) CometProject +Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#21] +Arguments: [ss_ext_sales_price#5], [ss_ext_sales_price#5] -(40) Project [codegen id : 7] -Output [1]: [ss_ext_sales_price#5] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#19] +(38) ColumnarToRow [codegen id : 1] +Input [1]: [ss_ext_sales_price#5] -(41) HashAggregate [codegen id : 7] +(39) HashAggregate [codegen id : 1] Input [1]: [ss_ext_sales_price#5] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#21] -Results [1]: [sum#22] +Aggregate Attributes [1]: [sum#23] +Results [1]: [sum#24] -(42) Exchange -Input [1]: [sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(40) Exchange +Input [1]: [sum#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(43) HashAggregate [codegen id : 15] -Input [1]: [sum#22] +(41) HashAggregate [codegen id : 4] +Input [1]: [sum#24] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#23,17,2) AS promotions#24] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#25] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#25,17,2) AS promotions#26] -(44) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +(42) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#27, ss_customer_sk#28, ss_store_sk#29, ss_ext_sales_price#30, ss_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(45) CometFilter -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) +(43) CometFilter +Input [5]: [ss_item_sk#27, ss_customer_sk#28, ss_store_sk#29, ss_ext_sales_price#30, ss_sold_date_sk#31] +Condition : ((isnotnull(ss_store_sk#29) AND isnotnull(ss_customer_sk#28)) AND isnotnull(ss_item_sk#27)) -(46) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +(44) ReusedExchange [Reuses operator id: 6] +Output [1]: [s_store_sk#33] -(47) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#31] +(45) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#27, ss_customer_sk#28, ss_store_sk#29, ss_ext_sales_price#30, ss_sold_date_sk#31] +Right output [1]: [s_store_sk#33] +Arguments: [ss_store_sk#29], [s_store_sk#33], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#27] -Right keys [1]: [s_store_sk#31] -Join type: Inner -Join condition: None +(46) CometProject +Input [6]: [ss_item_sk#27, ss_customer_sk#28, ss_store_sk#29, ss_ext_sales_price#30, ss_sold_date_sk#31, s_store_sk#33] +Arguments: [ss_item_sk#27, ss_customer_sk#28, ss_ext_sales_price#30, ss_sold_date_sk#31], [ss_item_sk#27, ss_customer_sk#28, ss_ext_sales_price#30, ss_sold_date_sk#31] -(49) Project [codegen id : 13] -Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] +(47) ReusedExchange [Reuses operator id: 18] +Output [1]: [d_date_sk#34] -(50) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#32] +(48) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#27, ss_customer_sk#28, ss_ext_sales_price#30, ss_sold_date_sk#31] +Right output [1]: [d_date_sk#34] +Arguments: [ss_sold_date_sk#31], [d_date_sk#34], Inner, BuildRight -(51) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#32] -Join type: Inner -Join condition: None +(49) CometProject +Input [5]: [ss_item_sk#27, ss_customer_sk#28, ss_ext_sales_price#30, ss_sold_date_sk#31, d_date_sk#34] +Arguments: [ss_item_sk#27, ss_customer_sk#28, ss_ext_sales_price#30], [ss_item_sk#27, ss_customer_sk#28, ss_ext_sales_price#30] -(52) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] +(50) ReusedExchange [Reuses operator id: 23] +Output [2]: [c_customer_sk#35, c_current_addr_sk#36] -(53) ReusedExchange [Reuses operator id: 24] -Output [2]: [c_customer_sk#33, c_current_addr_sk#34] +(51) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#27, ss_customer_sk#28, ss_ext_sales_price#30] +Right output [2]: [c_customer_sk#35, c_current_addr_sk#36] +Arguments: [ss_customer_sk#28], [c_customer_sk#35], Inner, BuildRight -(54) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#33] -Join type: Inner -Join condition: None +(52) CometProject +Input [5]: [ss_item_sk#27, ss_customer_sk#28, ss_ext_sales_price#30, c_customer_sk#35, c_current_addr_sk#36] +Arguments: [ss_item_sk#27, ss_ext_sales_price#30, c_current_addr_sk#36], [ss_item_sk#27, ss_ext_sales_price#30, c_current_addr_sk#36] -(55) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] +(53) ReusedExchange [Reuses operator id: 29] +Output [1]: [ca_address_sk#37] -(56) ReusedExchange [Reuses operator id: 31] -Output [1]: [ca_address_sk#35] +(54) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#27, ss_ext_sales_price#30, c_current_addr_sk#36] +Right output [1]: [ca_address_sk#37] +Arguments: [c_current_addr_sk#36], [ca_address_sk#37], Inner, BuildRight -(57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#35] -Join type: Inner -Join condition: None +(55) CometProject +Input [4]: [ss_item_sk#27, ss_ext_sales_price#30, c_current_addr_sk#36, ca_address_sk#37] +Arguments: [ss_item_sk#27, ss_ext_sales_price#30], [ss_item_sk#27, ss_ext_sales_price#30] -(58) Project [codegen id : 13] -Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] +(56) ReusedExchange [Reuses operator id: 35] +Output [1]: [i_item_sk#38] -(59) ReusedExchange [Reuses operator id: 38] -Output [1]: [i_item_sk#36] +(57) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#27, ss_ext_sales_price#30] +Right output [1]: [i_item_sk#38] +Arguments: [ss_item_sk#27], [i_item_sk#38], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#25] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None +(58) CometProject +Input [3]: [ss_item_sk#27, ss_ext_sales_price#30, i_item_sk#38] +Arguments: [ss_ext_sales_price#30], [ss_ext_sales_price#30] -(61) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#28] -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] +(59) ColumnarToRow [codegen id : 2] +Input [1]: [ss_ext_sales_price#30] -(62) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#28] +(60) HashAggregate [codegen id : 2] +Input [1]: [ss_ext_sales_price#30] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum#37] -Results [1]: [sum#38] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#30))] +Aggregate Attributes [1]: [sum#39] +Results [1]: [sum#40] -(63) Exchange -Input [1]: [sum#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(61) Exchange +Input [1]: [sum#40] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(64) HashAggregate [codegen id : 14] -Input [1]: [sum#38] +(62) HashAggregate [codegen id : 3] +Input [1]: [sum#40] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#28))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#39,17,2) AS total#40] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#30))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#30))#41] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#30))#41,17,2) AS total#42] -(65) BroadcastExchange -Input [1]: [total#40] -Arguments: IdentityBroadcastMode, [plan_id=8] +(63) BroadcastExchange +Input [1]: [total#42] +Arguments: IdentityBroadcastMode, [plan_id=3] -(66) BroadcastNestedLoopJoin [codegen id : 15] +(64) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(67) Project [codegen id : 15] -Output [3]: [promotions#24, total#40, ((cast(promotions#24 as decimal(15,4)) / cast(total#40 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#41] -Input [2]: [promotions#24, total#40] +(65) Project [codegen id : 4] +Output [3]: [promotions#26, total#42, ((cast(promotions#26 as decimal(15,4)) / cast(total#42 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#43] +Input [2]: [promotions#26, total#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (70) ++- * ColumnarToRow (69) + +- CometProject (68) + +- CometFilter (67) + +- CometScan parquet spark_catalog.default.date_dim (66) -(68) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#42, d_moy#43] +(66) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#14, d_year#15, d_moy#16] 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 -(69) CometFilter -Input [3]: [d_date_sk#14, d_year#42, d_moy#43] -Condition : ((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 1998)) AND (d_moy#43 = 11)) AND isnotnull(d_date_sk#14)) +(67) CometFilter +Input [3]: [d_date_sk#14, d_year#15, d_moy#16] +Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) -(70) CometProject -Input [3]: [d_date_sk#14, d_year#42, d_moy#43] +(68) CometProject +Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(71) ColumnarToRow [codegen id : 1] +(69) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(72) BroadcastExchange +(70) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 44 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt index 2c3d07ac64..db13f94bdf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt @@ -1,25 +1,25 @@ -WholeStageCodegen (15) +WholeStageCodegen (4) Project [promotions,total] BroadcastNestedLoopJoin HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] InputAdapter Exchange #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -30,77 +30,54 @@ WholeStageCodegen (15) CometProject [d_date_sk] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometBroadcastExchange #3 + CometProject [s_store_sk] + CometFilter [s_gmt_offset,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] + CometBroadcastExchange #4 + CometProject [p_promo_sk] + CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #7 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange #8 + CometProject [i_item_sk] + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (14) + BroadcastExchange #9 + WholeStageCodegen (3) HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] InputAdapter - Exchange #9 - WholeStageCodegen (13) + Exchange #10 + WholeStageCodegen (2) HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [s_store_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #6 - InputAdapter - ReusedExchange [i_item_sk] #7 + ReusedExchange [s_store_sk] #3 + ReusedExchange [d_date_sk] #5 + ReusedExchange [c_customer_sk,c_current_addr_sk] #6 + ReusedExchange [ca_address_sk] #7 + ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt index 0607d80770..cba9fc4b95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt @@ -1,36 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.ship_mode (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.web_site (16) - +- BroadcastExchange (26) - +- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +TakeOrderedAndProject (28) ++- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometProject (17) + : +- CometBroadcastHashJoin (16) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.warehouse (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.ship_mode (8) + : +- CometBroadcastExchange (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.web_site (13) + +- CometBroadcastExchange (21) + +- CometProject (20) + +- CometFilter (19) + +- CometScan parquet spark_catalog.default.date_dim (18) (1) Scan parquet spark_catalog.default.web_sales @@ -44,144 +40,128 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Condition : isnotnull(w_warehouse_sk#6) -(6) ColumnarToRow [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [w_warehouse_sk#6, w_warehouse_name#7] -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#4] -Right keys [1]: [w_warehouse_sk#6] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] +Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [ws_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight -(9) Project [codegen id : 5] -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] +(7) CometProject Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] -(10) Scan parquet spark_catalog.default.ship_mode +(8) Scan parquet spark_catalog.default.ship_mode Output [2]: [sm_ship_mode_sk#8, sm_type#9] Batched: true Location [not included in comparison]/{warehouse_dir}/ship_mode] PushedFilters: [IsNotNull(sm_ship_mode_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [sm_ship_mode_sk#8, sm_type#9] Condition : isnotnull(sm_ship_mode_sk#8) -(12) ColumnarToRow [codegen id : 2] -Input [2]: [sm_ship_mode_sk#8, sm_type#9] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [sm_ship_mode_sk#8, sm_type#9] -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_mode_sk#3] -Right keys [1]: [sm_ship_mode_sk#8] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] +Right output [2]: [sm_ship_mode_sk#8, sm_type#9] +Arguments: [ws_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight -(15) Project [codegen id : 5] -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9] +(12) CometProject Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#9] +Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9] -(16) Scan parquet spark_catalog.default.web_site +(13) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#10, web_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [2]: [web_site_sk#10, web_name#11] Condition : isnotnull(web_site_sk#10) -(18) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [2]: [web_site_sk#10, web_name#11] +Arguments: [web_site_sk#10, web_name#11] -(19) BroadcastExchange -Input [2]: [web_site_sk#10, web_name#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_web_site_sk#2] -Right keys [1]: [web_site_sk#10] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9] +Right output [2]: [web_site_sk#10, web_name#11] +Arguments: [ws_web_site_sk#2], [web_site_sk#10], Inner, BuildRight -(21) Project [codegen id : 5] -Output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11] +(17) CometProject Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_site_sk#10, web_name#11] +Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11], [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11] -(22) Scan parquet spark_catalog.default.date_dim +(18) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_month_seq#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(23) CometFilter +(19) CometFilter Input [2]: [d_date_sk#12, d_month_seq#13] Condition : (((isnotnull(d_month_seq#13) AND (d_month_seq#13 >= 1200)) AND (d_month_seq#13 <= 1211)) AND isnotnull(d_date_sk#12)) -(24) CometProject +(20) CometProject Input [2]: [d_date_sk#12, d_month_seq#13] Arguments: [d_date_sk#12], [d_date_sk#12] -(25) ColumnarToRow [codegen id : 4] -Input [1]: [d_date_sk#12] - -(26) BroadcastExchange +(21) CometBroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [d_date_sk#12] -(27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11] +Right output [1]: [d_date_sk#12] +Arguments: [ws_ship_date_sk#1], [d_date_sk#12], Inner, BuildRight -(28) Project [codegen id : 5] -Output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] +(23) CometProject Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11, d_date_sk#12] +Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _groupingexpression#14], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] + +(24) ColumnarToRow [codegen id : 1] +Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _groupingexpression#14] -(29) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 1] Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _groupingexpression#14] Keys [3]: [_groupingexpression#14, sm_type#9, web_name#11] Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] Results [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -(30) Exchange +(26) Exchange Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, web_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, web_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(31) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 2] Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [3]: [_groupingexpression#14, sm_type#9, web_name#11] Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] Aggregate Attributes [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] -(32) TakeOrderedAndProject +(28) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt index 5ae522ce1c..b25008e641 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt @@ -1,48 +1,32 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter Exchange [_groupingexpression,sm_type,web_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,w_warehouse_name] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - Project [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] + CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] + CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #2 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #3 + CometFilter [sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange #4 + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt index 990bc31957..c10d628aec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt @@ -1,32 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * Project (27) - +- * Filter (26) - +- Window (25) - +- * Sort (24) - +- Exchange (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.store (14) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * Sort (25) + +- Exchange (24) + +- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * ColumnarToRow (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- CometBroadcastExchange (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.store_sales (4) + : +- CometBroadcastExchange (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.date_dim (9) + +- CometBroadcastExchange (17) + +- CometFilter (16) + +- CometScan parquet spark_catalog.default.store (15) (1) Scan parquet spark_catalog.default.item @@ -44,10 +45,7 @@ Condition : ((((i_category#4 IN (Books Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] -(4) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#1, i_manager_id#5] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] @@ -55,140 +53,150 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) -(7) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [i_item_sk#1, i_manager_id#5] +Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight -(10) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +(8) CometProject Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] + +(9) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(10) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] +Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) + +(11) CometProject +Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] +Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(11) ReusedExchange [Reuses operator id: 33] -Output [2]: [d_date_sk#15, d_moy#16] +(12) CometBroadcastExchange +Input [2]: [d_date_sk#15, d_moy#17] +Arguments: [d_date_sk#15, d_moy#17] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Right output [2]: [d_date_sk#15, d_moy#17] +Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(13) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16] -Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#16] +(14) CometProject +Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#17] +Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] -(14) Scan parquet spark_catalog.default.store -Output [1]: [s_store_sk#17] +(15) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#18] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [1]: [s_store_sk#17] -Condition : isnotnull(s_store_sk#17) +(16) CometFilter +Input [1]: [s_store_sk#18] +Condition : isnotnull(s_store_sk#18) -(16) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#17] +(17) CometBroadcastExchange +Input [1]: [s_store_sk#18] +Arguments: [s_store_sk#18] -(17) BroadcastExchange -Input [1]: [s_store_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(18) CometBroadcastHashJoin +Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] +Right output [1]: [s_store_sk#18] +Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#17] -Join type: Inner -Join condition: None +(19) CometProject +Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17, s_store_sk#18] +Arguments: [i_manager_id#5, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_sales_price#12, d_moy#17] -(19) Project [codegen id : 4] -Output [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] -Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16, s_store_sk#17] +(20) ColumnarToRow [codegen id : 1] +Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#17] -(20) HashAggregate [codegen id : 4] -Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] -Keys [2]: [i_manager_id#5, d_moy#16] +(21) HashAggregate [codegen id : 1] +Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#17] +Keys [2]: [i_manager_id#5, d_moy#17] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [i_manager_id#5, d_moy#16, sum#19] +Aggregate Attributes [1]: [sum#19] +Results [3]: [i_manager_id#5, d_moy#17, sum#20] -(21) Exchange -Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) Exchange +Input [3]: [i_manager_id#5, d_moy#17, sum#20] +Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] -Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Keys [2]: [i_manager_id#5, d_moy#16] +(23) HashAggregate [codegen id : 2] +Input [3]: [i_manager_id#5, d_moy#17, sum#20] +Keys [2]: [i_manager_id#5, d_moy#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#21] +Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#21,17,2) AS sum_sales#22, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#21,17,2) AS _w0#23] -(23) Exchange -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(24) Exchange +Input [3]: [i_manager_id#5, sum_sales#22, _w0#23] +Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(24) Sort [codegen id : 6] -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +(25) Sort [codegen id : 3] +Input [3]: [i_manager_id#5, sum_sales#22, _w0#23] Arguments: [i_manager_id#5 ASC NULLS FIRST], false, 0 -(25) Window -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] +(26) Window +Input [3]: [i_manager_id#5, sum_sales#22, _w0#23] +Arguments: [avg(_w0#23) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#24], [i_manager_id#5] -(26) Filter [codegen id : 7] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END +(27) Filter [codegen id : 4] +Input [4]: [i_manager_id#5, sum_sales#22, _w0#23, avg_monthly_sales#24] +Condition : CASE WHEN (avg_monthly_sales#24 > 0.000000) THEN ((abs((sum_sales#22 - avg_monthly_sales#24)) / avg_monthly_sales#24) > 0.1000000000000000) ELSE false END -(27) Project [codegen id : 7] -Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] +(28) Project [codegen id : 4] +Output [3]: [i_manager_id#5, sum_sales#22, avg_monthly_sales#24] +Input [4]: [i_manager_id#5, sum_sales#22, _w0#23, avg_monthly_sales#24] -(28) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +(29) TakeOrderedAndProject +Input [3]: [i_manager_id#5, sum_sales#22, avg_monthly_sales#24] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#24 ASC NULLS FIRST, sum_sales#22 ASC NULLS FIRST], [i_manager_id#5, sum_sales#22, avg_monthly_sales#24] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) -(29) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] +(30) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(30) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) +(31) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] +Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(31) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] +(32) CometProject +Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] +Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(32) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_moy#16] +(33) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#15, d_moy#17] -(33) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(34) BroadcastExchange +Input [2]: [d_date_sk#15, d_moy#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt index 7f6f8c1370..a8eb9d7c6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt @@ -1,51 +1,45 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_manager_id] InputAdapter Exchange [i_manager_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_manager_id,d_moy] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - Project [i_manager_id,ss_sales_price,d_moy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometProject [i_item_sk,i_manager_id] CometFilter [i_category,i_class,i_brand,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt index 667362aa3c..d6e68a2acb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt @@ -216,7 +216,7 @@ Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_numbe (7) CometBroadcastHashJoin Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner +Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft (8) CometProject Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] @@ -725,7 +725,7 @@ Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_nu (117) CometBroadcastHashJoin Left output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Right output [2]: [sr_item_sk#119, sr_ticket_number#120] -Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner +Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner, BuildLeft (118) CometProject Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt index eda84bb525..ea79753667 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt @@ -1,43 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) +TakeOrderedAndProject (42) ++- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) : : :- * ColumnarToRow (3) : : : +- CometFilter (2) : : : +- CometScan parquet spark_catalog.default.store (1) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * HashAggregate (12) - : : +- Exchange (11) - : : +- * HashAggregate (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : +- ReusedExchange (7) - : +- BroadcastExchange (20) - : +- * ColumnarToRow (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - +- BroadcastExchange (36) - +- * Filter (35) - +- * HashAggregate (34) - +- Exchange (33) - +- * HashAggregate (32) - +- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * ColumnarToRow (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.store_sales (23) - +- ReusedExchange (26) + : : +- BroadcastExchange (17) + : : +- * Filter (16) + : : +- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * ColumnarToRow (12) + : : +- CometProject (11) + : : +- CometBroadcastHashJoin (10) + : : :- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : +- CometBroadcastExchange (9) + : : +- CometProject (8) + : : +- CometFilter (7) + : : +- CometScan parquet spark_catalog.default.date_dim (6) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- BroadcastExchange (39) + +- * Filter (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * HashAggregate (34) + +- Exchange (33) + +- * HashAggregate (32) + +- * ColumnarToRow (31) + +- CometProject (30) + +- CometBroadcastHashJoin (29) + :- CometFilter (27) + : +- CometScan parquet spark_catalog.default.store_sales (26) + +- ReusedExchange (28) (1) Scan parquet spark_catalog.default.store @@ -51,7 +54,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) ColumnarToRow [codegen id : 9] +(3) ColumnarToRow [codegen id : 7] Input [2]: [s_store_sk#1, s_store_name#2] (4) Scan parquet spark_catalog.default.store_sales @@ -66,204 +69,218 @@ ReadSchema: struct Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) -(6) ColumnarToRow [codegen id : 2] -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +(6) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#8, d_month_seq#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] +ReadSchema: struct -(7) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#8] +(7) CometFilter +Input [2]: [d_date_sk#8, d_month_seq#9] +Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] -Join type: Inner -Join condition: None +(8) CometProject +Input [2]: [d_date_sk#8, d_month_seq#9] +Arguments: [d_date_sk#8], [d_date_sk#8] -(9) Project [codegen id : 2] -Output [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] +(9) CometBroadcastExchange +Input [1]: [d_date_sk#8] +Arguments: [d_date_sk#8] + +(10) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +Right output [1]: [d_date_sk#8] +Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight + +(11) CometProject Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] +Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] + +(12) ColumnarToRow [codegen id : 1] +Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -(10) HashAggregate [codegen id : 2] +(13) HashAggregate [codegen id : 1] Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum#9] -Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] +Aggregate Attributes [1]: [sum#10] +Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#11] -(11) Exchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] +(14) Exchange +Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#11] Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(12) HashAggregate [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] +(15) HashAggregate [codegen id : 2] +Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#11] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] -Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#12] +Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#12,17,2) AS revenue#13] -(13) Filter [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Condition : isnotnull(revenue#12) +(16) Filter [codegen id : 2] +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#13] +Condition : isnotnull(revenue#13) -(14) BroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] +(17) BroadcastExchange +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 9] +(18) BroadcastHashJoin [codegen id : 7] Left keys [1]: [s_store_sk#1] Right keys [1]: [ss_store_sk#4] Join type: Inner Join condition: None -(16) Project [codegen id : 9] -Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] +(19) Project [codegen id : 7] +Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#13] +Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#13] -(17) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +(20) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#14, i_item_desc#15, i_current_price#16, i_wholesale_cost#17, i_brand#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(18) CometFilter -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Condition : isnotnull(i_item_sk#13) +(21) CometFilter +Input [5]: [i_item_sk#14, i_item_desc#15, i_current_price#16, i_wholesale_cost#17, i_brand#18] +Condition : isnotnull(i_item_sk#14) -(19) ColumnarToRow [codegen id : 4] -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +(22) ColumnarToRow [codegen id : 3] +Input [5]: [i_item_sk#14, i_item_desc#15, i_current_price#16, i_wholesale_cost#17, i_brand#18] -(20) BroadcastExchange -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +(23) BroadcastExchange +Input [5]: [i_item_sk#14, i_item_desc#15, i_current_price#16, i_wholesale_cost#17, i_brand#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(21) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#3] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(22) Project [codegen id : 9] -Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +(25) Project [codegen id : 7] +Output [7]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#15, i_current_price#16, i_wholesale_cost#17, i_brand#18] +Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#13, i_item_sk#14, i_item_desc#15, i_current_price#16, i_wholesale_cost#17, i_brand#18] -(23) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +(26) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(24) CometFilter -Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_store_sk#19) +(27) CometFilter +Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_store_sk#20) -(25) ColumnarToRow [codegen id : 6] -Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +(28) ReusedExchange [Reuses operator id: 9] +Output [1]: [d_date_sk#24] -(26) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#23] +(29) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] +Right output [1]: [d_date_sk#24] +Arguments: [ss_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#21] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None +(30) CometProject +Input [5]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#24] +Arguments: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21], [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] + +(31) ColumnarToRow [codegen id : 4] +Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] -(28) Project [codegen id : 6] -Output [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] - -(29) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] - -(30) Exchange -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] -Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#20))#26] -Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#26,17,2) AS revenue#27] - -(32) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#19, revenue#27] -Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#27)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ss_store_sk#19, sum#30, count#31] +(32) HashAggregate [codegen id : 4] +Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] +Keys [2]: [ss_store_sk#20, ss_item_sk#19] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] +Aggregate Attributes [1]: [sum#25] +Results [3]: [ss_store_sk#20, ss_item_sk#19, sum#26] (33) Exchange -Input [3]: [ss_store_sk#19, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(34) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#19, sum#30, count#31] -Keys [1]: [ss_store_sk#19] -Functions [1]: [avg(revenue#27)] -Aggregate Attributes [1]: [avg(revenue#27)#32] -Results [2]: [ss_store_sk#19, avg(revenue#27)#32 AS ave#33] - -(35) Filter [codegen id : 8] -Input [2]: [ss_store_sk#19, ave#33] -Condition : isnotnull(ave#33) - -(36) BroadcastExchange -Input [2]: [ss_store_sk#19, ave#33] +Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#26] +Arguments: hashpartitioning(ss_store_sk#20, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(34) HashAggregate [codegen id : 5] +Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#26] +Keys [2]: [ss_store_sk#20, ss_item_sk#19] +Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#21))#27] +Results [2]: [ss_store_sk#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#21))#27,17,2) AS revenue#28] + +(35) HashAggregate [codegen id : 5] +Input [2]: [ss_store_sk#20, revenue#28] +Keys [1]: [ss_store_sk#20] +Functions [1]: [partial_avg(revenue#28)] +Aggregate Attributes [2]: [sum#29, count#30] +Results [3]: [ss_store_sk#20, sum#31, count#32] + +(36) Exchange +Input [3]: [ss_store_sk#20, sum#31, count#32] +Arguments: hashpartitioning(ss_store_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(37) HashAggregate [codegen id : 6] +Input [3]: [ss_store_sk#20, sum#31, count#32] +Keys [1]: [ss_store_sk#20] +Functions [1]: [avg(revenue#28)] +Aggregate Attributes [1]: [avg(revenue#28)#33] +Results [2]: [ss_store_sk#20, avg(revenue#28)#33 AS ave#34] + +(38) Filter [codegen id : 6] +Input [2]: [ss_store_sk#20, ave#34] +Condition : isnotnull(ave#34) + +(39) BroadcastExchange +Input [2]: [ss_store_sk#20, ave#34] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#19] +Right keys [1]: [ss_store_sk#20] Join type: Inner -Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) +Join condition: (cast(revenue#13 as decimal(23,7)) <= (0.1 * ave#34)) -(38) Project [codegen id : 9] -Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17, ss_store_sk#19, ave#33] +(41) Project [codegen id : 7] +Output [6]: [s_store_name#2, i_item_desc#15, revenue#13, i_current_price#16, i_wholesale_cost#17, i_brand#18] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#15, i_current_price#16, i_wholesale_cost#17, i_brand#18, ss_store_sk#20, ave#34] -(39) TakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] +(42) TakeOrderedAndProject +Input [6]: [s_store_name#2, i_item_desc#15, revenue#13, i_current_price#16, i_wholesale_cost#17, i_brand#18] +Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#15 ASC NULLS FIRST], [s_store_name#2, i_item_desc#15, revenue#13, i_current_price#16, i_wholesale_cost#17, i_brand#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(40) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#34] +(43) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#8, d_month_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_month_seq#34 <= 1187)) AND isnotnull(d_date_sk#8)) +(44) CometFilter +Input [2]: [d_date_sk#8, d_month_seq#9] +Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) -(42) CometProject -Input [2]: [d_date_sk#8, d_month_seq#34] +(45) CometProject +Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(43) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(44) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt index 33b695e811..1e34ce22f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (9) + WholeStageCodegen (7) Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] @@ -12,17 +12,17 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [revenue] HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter Exchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -33,35 +33,36 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (6) Filter [ave] HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] InputAdapter - Exchange [ss_store_sk] #6 - WholeStageCodegen (7) + Exchange [ss_store_sk] #7 + WholeStageCodegen (5) HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter - Exchange [ss_store_sk,ss_item_sk] #7 - WholeStageCodegen (6) + Exchange [ss_store_sk,ss_item_sk] #8 + WholeStageCodegen (4) HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt index a26c457aa4..ea11cbb749 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt @@ -1,56 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- Union (48) - :- * HashAggregate (29) - : +- Exchange (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.time_dim (13) - : +- BroadcastExchange (24) - : +- * ColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.ship_mode (20) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (32) - : : : : +- CometFilter (31) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- ReusedExchange (42) +TakeOrderedAndProject (51) ++- * HashAggregate (50) + +- Exchange (49) + +- * HashAggregate (48) + +- Union (47) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * ColumnarToRow (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.warehouse (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.time_dim (13) + : +- CometBroadcastExchange (22) + : +- CometProject (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.ship_mode (19) + +- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * ColumnarToRow (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- ReusedExchange (40) (1) Scan parquet spark_catalog.default.web_sales @@ -65,49 +64,53 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] Condition : isnotnull(w_warehouse_sk#9) -(6) ColumnarToRow [codegen id : 1] -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#3] -Right keys [1]: [w_warehouse_sk#9] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] +Right output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Arguments: [ws_warehouse_sk#3], [w_warehouse_sk#9], Inner, BuildRight -(9) Project [codegen id : 5] -Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +(7) CometProject Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -(10) ReusedExchange [Reuses operator id: 56] +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#16, d_year#17, d_moy#18] +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 + +(9) CometFilter +Input [3]: [d_date_sk#16, d_year#17, d_moy#18] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#7] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(10) CometBroadcastExchange +Input [3]: [d_date_sk#16, d_year#17, d_moy#18] +Arguments: [d_date_sk#16, d_year#17, d_moy#18] -(12) Project [codegen id : 5] -Output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +(11) CometBroadcastHashJoin +Left output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Right output [3]: [d_date_sk#16, d_year#17, d_moy#18] +Arguments: [ws_sold_date_sk#7], [d_date_sk#16], Inner, BuildRight + +(12) CometProject Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_date_sk#16, d_year#17, d_moy#18] +Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] (13) Scan parquet spark_catalog.default.time_dim Output [2]: [t_time_sk#19, t_time#20] @@ -124,74 +127,69 @@ Condition : (((isnotnull(t_time#20) AND (t_time#20 >= 30838)) AND (t_time#20 <= Input [2]: [t_time_sk#19, t_time#20] Arguments: [t_time_sk#19], [t_time_sk#19] -(16) ColumnarToRow [codegen id : 3] -Input [1]: [t_time_sk#19] - -(17) BroadcastExchange +(16) CometBroadcastExchange Input [1]: [t_time_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [t_time_sk#19] -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#19] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +Right output [1]: [t_time_sk#19] +Arguments: [ws_sold_time_sk#1], [t_time_sk#19], Inner, BuildRight -(19) Project [codegen id : 5] -Output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +(18) CometProject Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18, t_time_sk#19] +Arguments: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18], [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] -(20) Scan parquet spark_catalog.default.ship_mode +(19) Scan parquet spark_catalog.default.ship_mode Output [2]: [sm_ship_mode_sk#21, sm_carrier#22] Batched: true Location [not included in comparison]/{warehouse_dir}/ship_mode] PushedFilters: [In(sm_carrier, [BARIAN ,DHL ]), IsNotNull(sm_ship_mode_sk)] ReadSchema: struct -(21) CometFilter +(20) CometFilter Input [2]: [sm_ship_mode_sk#21, sm_carrier#22] Condition : (sm_carrier#22 IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#21)) -(22) CometProject +(21) CometProject Input [2]: [sm_ship_mode_sk#21, sm_carrier#22] Arguments: [sm_ship_mode_sk#21], [sm_ship_mode_sk#21] -(23) ColumnarToRow [codegen id : 4] +(22) CometBroadcastExchange Input [1]: [sm_ship_mode_sk#21] +Arguments: [sm_ship_mode_sk#21] -(24) BroadcastExchange -Input [1]: [sm_ship_mode_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_mode_sk#2] -Right keys [1]: [sm_ship_mode_sk#21] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +Right output [1]: [sm_ship_mode_sk#21] +Arguments: [ws_ship_mode_sk#2], [sm_ship_mode_sk#21], Inner, BuildRight -(26) Project [codegen id : 5] -Output [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +(24) CometProject Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18, sm_ship_mode_sk#21] +Arguments: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18], [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] -(27) HashAggregate [codegen id : 5] +(25) ColumnarToRow [codegen id : 1] +Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] + +(26) HashAggregate [codegen id : 1] Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17] Functions [24]: [partial_sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] Aggregate Attributes [48]: [sum#23, isEmpty#24, sum#25, isEmpty#26, sum#27, isEmpty#28, sum#29, isEmpty#30, sum#31, isEmpty#32, sum#33, isEmpty#34, sum#35, isEmpty#36, sum#37, isEmpty#38, sum#39, isEmpty#40, sum#41, isEmpty#42, sum#43, isEmpty#44, sum#45, isEmpty#46, sum#47, isEmpty#48, sum#49, isEmpty#50, sum#51, isEmpty#52, sum#53, isEmpty#54, sum#55, isEmpty#56, sum#57, isEmpty#58, sum#59, isEmpty#60, sum#61, isEmpty#62, sum#63, isEmpty#64, sum#65, isEmpty#66, sum#67, isEmpty#68, sum#69, isEmpty#70] Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] -(28) Exchange +(27) Exchange Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 2] Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17] Functions [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142] Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#143, d_year#17 AS year#144, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119 AS jan_sales#145, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120 AS feb_sales#146, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121 AS mar_sales#147, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122 AS apr_sales#148, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123 AS may_sales#149, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124 AS jun_sales#150, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125 AS jul_sales#151, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126 AS aug_sales#152, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127 AS sep_sales#153, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128 AS oct_sales#154, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129 AS nov_sales#155, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130 AS dec_sales#156, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131 AS jan_net#157, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132 AS feb_net#158, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133 AS mar_net#159, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134 AS apr_net#160, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135 AS may_net#161, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136 AS jun_net#162, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137 AS jul_net#163, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138 AS aug_net#164, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139 AS sep_net#165, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140 AS oct_net#166, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141 AS nov_net#167, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142 AS dec_net#168] -(30) Scan parquet spark_catalog.default.catalog_sales +(29) Scan parquet spark_catalog.default.catalog_sales Output [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] Batched: true Location: InMemoryFileIndex [] @@ -199,134 +197,130 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#175), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(31) CometFilter +(30) CometFilter Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] Condition : ((isnotnull(cs_warehouse_sk#171) AND isnotnull(cs_sold_time_sk#169)) AND isnotnull(cs_ship_mode_sk#170)) -(32) ColumnarToRow [codegen id : 11] -Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] - -(33) ReusedExchange [Reuses operator id: 7] +(31) ReusedExchange [Reuses operator id: 5] Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_warehouse_sk#171] -Right keys [1]: [w_warehouse_sk#177] -Join type: Inner -Join condition: None +(32) CometBroadcastHashJoin +Left output [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] +Right output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] +Arguments: [cs_warehouse_sk#171], [w_warehouse_sk#177], Inner, BuildRight -(35) Project [codegen id : 11] -Output [12]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] +(33) CometProject Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] +Arguments: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183], [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -(36) ReusedExchange [Reuses operator id: 56] +(34) ReusedExchange [Reuses operator id: 10] Output [3]: [d_date_sk#184, d_year#185, d_moy#186] -(37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#175] -Right keys [1]: [d_date_sk#184] -Join type: Inner -Join condition: None +(35) CometBroadcastHashJoin +Left output [12]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] +Right output [3]: [d_date_sk#184, d_year#185, d_moy#186] +Arguments: [cs_sold_date_sk#175], [d_date_sk#184], Inner, BuildRight -(38) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +(36) CometProject Input [15]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] +Arguments: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186], [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -(39) ReusedExchange [Reuses operator id: 17] +(37) ReusedExchange [Reuses operator id: 16] Output [1]: [t_time_sk#187] -(40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_time_sk#169] -Right keys [1]: [t_time_sk#187] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [13]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +Right output [1]: [t_time_sk#187] +Arguments: [cs_sold_time_sk#169], [t_time_sk#187], Inner, BuildRight -(41) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +(39) CometProject Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] +Arguments: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186], [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -(42) ReusedExchange [Reuses operator id: 24] +(40) ReusedExchange [Reuses operator id: 22] Output [1]: [sm_ship_mode_sk#188] -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_ship_mode_sk#170] -Right keys [1]: [sm_ship_mode_sk#188] -Join type: Inner -Join condition: None +(41) CometBroadcastHashJoin +Left output [12]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +Right output [1]: [sm_ship_mode_sk#188] +Arguments: [cs_ship_mode_sk#170], [sm_ship_mode_sk#188], Inner, BuildRight -(44) Project [codegen id : 11] -Output [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +(42) CometProject Input [13]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] +Arguments: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186], [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] + +(43) ColumnarToRow [codegen id : 3] +Input [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -(45) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 3] Input [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] Aggregate Attributes [48]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236] Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -(46) Exchange +(45) Exchange Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(47) HashAggregate [codegen id : 12] +(46) HashAggregate [codegen id : 4] Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308] Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, DHL,BARIAN AS ship_carriers#309, d_year#185 AS year#310, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285 AS jan_sales#311, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286 AS feb_sales#312, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287 AS mar_sales#313, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288 AS apr_sales#314, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289 AS may_sales#315, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290 AS jun_sales#316, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291 AS jul_sales#317, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292 AS aug_sales#318, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293 AS sep_sales#319, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294 AS oct_sales#320, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295 AS nov_sales#321, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296 AS dec_sales#322, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297 AS jan_net#323, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298 AS feb_net#324, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299 AS mar_net#325, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300 AS apr_net#326, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301 AS may_net#327, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302 AS jun_net#328, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303 AS jul_net#329, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304 AS aug_net#330, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305 AS sep_net#331, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306 AS oct_net#332, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307 AS nov_net#333, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308 AS dec_net#334] -(48) Union +(47) Union -(49) HashAggregate [codegen id : 13] +(48) HashAggregate [codegen id : 5] Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#145, feb_sales#146, mar_sales#147, apr_sales#148, may_sales#149, jun_sales#150, jul_sales#151, aug_sales#152, sep_sales#153, oct_sales#154, nov_sales#155, dec_sales#156, jan_net#157, feb_net#158, mar_net#159, apr_net#160, may_net#161, jun_net#162, jul_net#163, aug_net#164, sep_net#165, oct_net#166, nov_net#167, dec_net#168] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial_sum(mar_sales#147), partial_sum(apr_sales#148), partial_sum(may_sales#149), partial_sum(jun_sales#150), partial_sum(jul_sales#151), partial_sum(aug_sales#152), partial_sum(sep_sales#153), partial_sum(oct_sales#154), partial_sum(nov_sales#155), partial_sum(dec_sales#156), partial_sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#157), partial_sum(feb_net#158), partial_sum(mar_net#159), partial_sum(apr_net#160), partial_sum(may_net#161), partial_sum(jun_net#162), partial_sum(jul_net#163), partial_sum(aug_net#164), partial_sum(sep_net#165), partial_sum(oct_net#166), partial_sum(nov_net#167), partial_sum(dec_net#168)] Aggregate Attributes [72]: [sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384, sum#385, isEmpty#386, sum#387, isEmpty#388, sum#389, isEmpty#390, sum#391, isEmpty#392, sum#393, isEmpty#394, sum#395, isEmpty#396, sum#397, isEmpty#398, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406] Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -(50) Exchange +(49) Exchange Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(51) HashAggregate [codegen id : 14] +(50) HashAggregate [codegen id : 6] Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#157), sum(feb_net#158), sum(mar_net#159), sum(apr_net#160), sum(may_net#161), sum(jun_net#162), sum(jul_net#163), sum(aug_net#164), sum(sep_net#165), sum(oct_net#166), sum(nov_net#167), sum(dec_net#168)] Aggregate Attributes [36]: [sum(jan_sales#145)#479, sum(feb_sales#146)#480, sum(mar_sales#147)#481, sum(apr_sales#148)#482, sum(may_sales#149)#483, sum(jun_sales#150)#484, sum(jul_sales#151)#485, sum(aug_sales#152)#486, sum(sep_sales#153)#487, sum(oct_sales#154)#488, sum(nov_sales#155)#489, sum(dec_sales#156)#490, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502, sum(jan_net#157)#503, sum(feb_net#158)#504, sum(mar_net#159)#505, sum(apr_net#160)#506, sum(may_net#161)#507, sum(jun_net#162)#508, sum(jul_net#163)#509, sum(aug_net#164)#510, sum(sep_net#165)#511, sum(oct_net#166)#512, sum(nov_net#167)#513, sum(dec_net#168)#514] Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum(jan_sales#145)#479 AS jan_sales#515, sum(feb_sales#146)#480 AS feb_sales#516, sum(mar_sales#147)#481 AS mar_sales#517, sum(apr_sales#148)#482 AS apr_sales#518, sum(may_sales#149)#483 AS may_sales#519, sum(jun_sales#150)#484 AS jun_sales#520, sum(jul_sales#151)#485 AS jul_sales#521, sum(aug_sales#152)#486 AS aug_sales#522, sum(sep_sales#153)#487 AS sep_sales#523, sum(oct_sales#154)#488 AS oct_sales#524, sum(nov_sales#155)#489 AS nov_sales#525, sum(dec_sales#156)#490 AS dec_sales#526, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491 AS jan_sales_per_sq_foot#527, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492 AS feb_sales_per_sq_foot#528, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493 AS mar_sales_per_sq_foot#529, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494 AS apr_sales_per_sq_foot#530, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495 AS may_sales_per_sq_foot#531, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496 AS jun_sales_per_sq_foot#532, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497 AS jul_sales_per_sq_foot#533, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498 AS aug_sales_per_sq_foot#534, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499 AS sep_sales_per_sq_foot#535, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500 AS oct_sales_per_sq_foot#536, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501 AS nov_sales_per_sq_foot#537, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502 AS dec_sales_per_sq_foot#538, sum(jan_net#157)#503 AS jan_net#539, sum(feb_net#158)#504 AS feb_net#540, sum(mar_net#159)#505 AS mar_net#541, sum(apr_net#160)#506 AS apr_net#542, sum(may_net#161)#507 AS may_net#543, sum(jun_net#162)#508 AS jun_net#544, sum(jul_net#163)#509 AS jul_net#545, sum(aug_net#164)#510 AS aug_net#546, sum(sep_net#165)#511 AS sep_net#547, sum(oct_net#166)#512 AS oct_net#548, sum(nov_net#167)#513 AS nov_net#549, sum(dec_net#168)#514 AS dec_net#550] -(52) TakeOrderedAndProject +(51) TakeOrderedAndProject Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (56) -+- * ColumnarToRow (55) - +- CometFilter (54) - +- CometScan parquet spark_catalog.default.date_dim (53) +BroadcastExchange (55) ++- * ColumnarToRow (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) -(53) Scan parquet spark_catalog.default.date_dim +(52) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#16, d_year#17, d_moy#18] 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 -(54) CometFilter +(53) CometFilter Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(55) ColumnarToRow [codegen id : 1] +(54) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -(56) BroadcastExchange +(55) BroadcastExchange Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#175 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#175 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt index 8ed74582f0..7d6c85af33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt @@ -1,28 +1,28 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) + WholeStageCodegen (6) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) + WholeStageCodegen (5) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -32,55 +32,40 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) + CometBroadcastExchange #4 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometBroadcastExchange #5 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometProject [t_time_sk] + CometFilter [t_time,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + CometBroadcastExchange #7 + CometProject [sm_ship_mode_sk] + CometFilter [sm_carrier,sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 + WholeStageCodegen (3) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - ReusedExchange [t_time_sk] #5 - InputAdapter - ReusedExchange [sm_ship_mode_sk] #6 + ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + ReusedExchange [d_date_sk,d_year,d_moy] #5 + ReusedExchange [t_time_sk] #6 + ReusedExchange [sm_ship_mode_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt index d1a6a4f471..a1624de1c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt @@ -1,34 +1,35 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Filter (29) - +- Window (28) - +- WindowGroupLimit (27) - +- * Sort (26) - +- Exchange (25) - +- WindowGroupLimit (24) - +- * Sort (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.item (13) +TakeOrderedAndProject (31) ++- * Filter (30) + +- Window (29) + +- WindowGroupLimit (28) + +- * Sort (27) + +- Exchange (26) + +- WindowGroupLimit (25) + +- * Sort (24) + +- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * ColumnarToRow (20) + +- CometExpand (19) + +- CometProject (18) + +- CometBroadcastHashJoin (17) + :- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.store (9) + +- CometBroadcastExchange (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.item (14) (1) Scan parquet spark_catalog.default.store_sales @@ -43,162 +44,169 @@ ReadSchema: struct + +(4) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) -(4) ReusedExchange [Reuses operator id: 35] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(5) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(6) CometBroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] +Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] +(8) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] + +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#12, s_store_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) - -(9) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#12] +(10) CometFilter +Input [2]: [s_store_sk#12, s_store_id#13] +Condition : isnotnull(s_store_sk#12) -(10) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [2]: [s_store_sk#12, s_store_id#13] +Arguments: [s_store_sk#12, s_store_id#13] -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] +Right output [2]: [s_store_sk#12, s_store_id#13] +Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight -(12) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#12] +(13) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#13] +Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13] -(13) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +(14) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(14) CometFilter -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Condition : isnotnull(i_item_sk#13) +(15) CometFilter +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Condition : isnotnull(i_item_sk#14) -(15) ColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +(16) CometBroadcastExchange +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -(16) BroadcastExchange -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13] +Right output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [ss_item_sk#1], [i_item_sk#14], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(18) CometProject +Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] -(18) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +(19) CometExpand +Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] +Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, 0], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#17, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] -(19) Expand [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] -Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 0], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#16, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] +(20) ColumnarToRow [codegen id : 1] +Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] -(20) HashAggregate [codegen id : 4] -Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] -Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] +(21) HashAggregate [codegen id : 1] +Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] +Keys [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#27, isEmpty#28] -Results [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] +Aggregate Attributes [2]: [sum#28, isEmpty#29] +Results [11]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27, sum#30, isEmpty#31] -(21) Exchange -Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] -Arguments: hashpartitioning(i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) Exchange +Input [11]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27, sum#30, isEmpty#31] +Arguments: hashpartitioning(i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] -Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] -Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] +(23) HashAggregate [codegen id : 2] +Input [11]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27, sum#30, isEmpty#31] +Keys [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31] -Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31 AS sumsales#32] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#32] +Results [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#32 AS sumsales#33] -(23) Sort [codegen id : 5] -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0 +(24) Sort [codegen id : 2] +Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#33] +Arguments: [i_category#19 ASC NULLS FIRST, sumsales#33 DESC NULLS LAST], false, 0 -(24) WindowGroupLimit -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: [i_category#18], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Partial +(25) WindowGroupLimit +Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#33] +Arguments: [i_category#19], [sumsales#33 DESC NULLS LAST], rank(sumsales#33), 100, Partial -(25) Exchange -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(26) Exchange +Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#33] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(26) Sort [codegen id : 6] -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0 +(27) Sort [codegen id : 3] +Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#33] +Arguments: [i_category#19 ASC NULLS FIRST, sumsales#33 DESC NULLS LAST], false, 0 -(27) WindowGroupLimit -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: [i_category#18], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Final +(28) WindowGroupLimit +Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#33] +Arguments: [i_category#19], [sumsales#33 DESC NULLS LAST], rank(sumsales#33), 100, Final -(28) Window -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: [rank(sumsales#32) windowspecdefinition(i_category#18, sumsales#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#33], [i_category#18], [sumsales#32 DESC NULLS LAST] +(29) Window +Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#33] +Arguments: [rank(sumsales#33) windowspecdefinition(i_category#19, sumsales#33 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [i_category#19], [sumsales#33 DESC NULLS LAST] -(29) Filter [codegen id : 7] -Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] -Condition : (rk#33 <= 100) +(30) Filter [codegen id : 4] +Input [10]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#33, rk#34] +Condition : (rk#34 <= 100) -(30) TakeOrderedAndProject -Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] -Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_qoy#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, sumsales#32 ASC NULLS FIRST, rk#33 ASC NULLS FIRST], [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] +(31) TakeOrderedAndProject +Input [10]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#33, rk#34] +Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, i_product_name#22 ASC NULLS FIRST, d_year#23 ASC NULLS FIRST, d_qoy#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, s_store_id#26 ASC NULLS FIRST, sumsales#33 ASC NULLS FIRST, rk#34 ASC NULLS FIRST], [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#33, rk#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (36) ++- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(31) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] +(32) Scan parquet spark_catalog.default.date_dim +Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#7)) +(33) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) -(33) CometProject -Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(34) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(34) ColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(35) ColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(35) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) BroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt index e10def3975..0386feff9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt @@ -1,30 +1,30 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [rk] InputAdapter Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,sumsales] InputAdapter Exchange [i_category] #1 WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (5) + WholeStageCodegen (2) Sort [i_category,sumsales] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] + CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -35,19 +35,13 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange #5 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt index 734b6c11c6..850cd5ccb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt @@ -1,43 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) +TakeOrderedAndProject (42) ++- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) :- * Project (35) : +- * BroadcastHashJoin Inner BuildRight (34) : :- * HashAggregate (29) : : +- Exchange (28) : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) + : : +- * ColumnarToRow (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometProject (20) + : : : +- CometBroadcastHashJoin (19) + : : : :- CometProject (14) + : : : : +- CometBroadcastHashJoin (13) + : : : : :- CometProject (8) + : : : : : +- CometBroadcastHashJoin (7) + : : : : : :- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- CometBroadcastExchange (6) + : : : : : +- CometProject (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : : +- CometBroadcastExchange (12) + : : : : +- CometProject (11) + : : : : +- CometFilter (10) + : : : : +- CometScan parquet spark_catalog.default.store (9) + : : : +- CometBroadcastExchange (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.household_demographics (15) + : : +- CometBroadcastExchange (23) : : +- CometFilter (22) : : +- CometScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (33) : +- * ColumnarToRow (32) : +- CometFilter (31) : +- CometScan parquet spark_catalog.default.customer (30) - +- ReusedExchange (36) + +- BroadcastExchange (39) + +- * ColumnarToRow (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.customer_address (36) (1) Scan parquet spark_catalog.default.store_sales @@ -52,207 +55,225 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#11] +(4) CometFilter +Input [3]: [d_date_sk#11, d_year#12, d_dom#13] +Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#11, d_year#12, d_dom#13] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(7) CometBroadcastHashJoin +Left output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] +Right output [1]: [d_date_sk#11] +Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight -(6) Project [codegen id : 5] -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] +(8) CometProject Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_city#13] +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#14, s_city#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#12, s_city#13] -Condition : (s_city#13 IN (Midway,Fairview) AND isnotnull(s_store_sk#12)) +(10) CometFilter +Input [2]: [s_store_sk#14, s_city#15] +Condition : (s_city#15 IN (Midway,Fairview) AND isnotnull(s_store_sk#14)) -(9) CometProject -Input [2]: [s_store_sk#12, s_city#13] -Arguments: [s_store_sk#12], [s_store_sk#12] +(11) CometProject +Input [2]: [s_store_sk#14, s_city#15] +Arguments: [s_store_sk#14], [s_store_sk#14] -(10) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#12] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#14] +Arguments: [s_store_sk#14] -(11) BroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] +Right output [1]: [s_store_sk#14] +Arguments: [ss_store_sk#4], [s_store_sk#14], Inner, BuildRight -(13) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#12] +(14) CometProject +Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#14] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -(14) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] +(15) Scan parquet spark_catalog.default.household_demographics +Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : (((hd_dep_count#15 = 4) OR (hd_vehicle_count#16 = 3)) AND isnotnull(hd_demo_sk#14)) +(16) CometFilter +Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] +Condition : (((hd_dep_count#17 = 4) OR (hd_vehicle_count#18 = 3)) AND isnotnull(hd_demo_sk#16)) -(16) CometProject -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Arguments: [hd_demo_sk#14], [hd_demo_sk#14] +(17) CometProject +Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] +Arguments: [hd_demo_sk#16], [hd_demo_sk#16] -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#14] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#16] +Arguments: [hd_demo_sk#16] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(19) CometBroadcastHashJoin +Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] +Right output [1]: [hd_demo_sk#16] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#14] +(20) CometProject +Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#16] +Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] (21) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_city#18] +Output [2]: [ca_address_sk#19, ca_city#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] ReadSchema: struct (22) CometFilter -Input [2]: [ca_address_sk#17, ca_city#18] -Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_city#18)) +Input [2]: [ca_address_sk#19, ca_city#20] +Condition : (isnotnull(ca_address_sk#19) AND isnotnull(ca_city#20)) -(23) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#17, ca_city#18] +(23) CometBroadcastExchange +Input [2]: [ca_address_sk#19, ca_city#20] +Arguments: [ca_address_sk#19, ca_city#20] -(24) BroadcastExchange -Input [2]: [ca_address_sk#17, ca_city#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(24) CometBroadcastHashJoin +Left output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] +Right output [2]: [ca_address_sk#19, ca_city#20] +Arguments: [ss_addr_sk#3], [ca_address_sk#19], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None +(25) CometProject +Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#19, ca_city#20] +Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] -(26) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] -Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#17, ca_city#18] +(26) ColumnarToRow [codegen id : 1] +Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] -(27) HashAggregate [codegen id : 5] -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] +(27) HashAggregate [codegen id : 1] +Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum#19, sum#20, sum#21] -Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] +Aggregate Attributes [3]: [sum#21, sum#22, sum#23] +Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#24, sum#25, sum#26] (28) Exchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#24, sum#25, sum#26] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 8] -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] +(29) HashAggregate [codegen id : 4] +Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#24, sum#25, sum#26] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#25, sum(UnscaledValue(ss_ext_list_price#7))#26, sum(UnscaledValue(ss_ext_tax#8))#27] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#18 AS bought_city#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#25,17,2) AS extended_price#29, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#26,17,2) AS list_price#30, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#27,17,2) AS extended_tax#31] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#27, sum(UnscaledValue(ss_ext_list_price#7))#28, sum(UnscaledValue(ss_ext_tax#8))#29] +Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#20 AS bought_city#30, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#27,17,2) AS extended_price#31, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#28,17,2) AS list_price#32, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#29,17,2) AS extended_tax#33] (30) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +Output [4]: [c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (31) CometFilter -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) +Input [4]: [c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] +Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#35)) -(32) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +(32) ColumnarToRow [codegen id : 2] +Input [4]: [c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] (33) BroadcastExchange -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Input [4]: [c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(34) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#32] +Right keys [1]: [c_customer_sk#34] Join type: Inner Join condition: None -(35) Project [codegen id : 8] -Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +(35) Project [codegen id : 4] +Output [8]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37] +Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] + +(36) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#38, ca_city#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] +ReadSchema: struct -(36) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#36, ca_city#37] +(37) CometFilter +Input [2]: [ca_address_sk#38, ca_city#39] +Condition : (isnotnull(ca_address_sk#38) AND isnotnull(ca_city#39)) + +(38) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#38, ca_city#39] + +(39) BroadcastExchange +Input [2]: [ca_address_sk#38, ca_city#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#36] +(40) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [c_current_addr_sk#35] +Right keys [1]: [ca_address_sk#38] Join type: Inner -Join condition: NOT (ca_city#37 = bought_city#28) +Join condition: NOT (ca_city#39 = bought_city#30) -(38) Project [codegen id : 8] -Output [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35, ca_address_sk#36, ca_city#37] +(41) Project [codegen id : 4] +Output [8]: [c_last_name#37, c_first_name#36, ca_city#39, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] +Input [10]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#38, ca_city#39] -(39) TakeOrderedAndProject -Input [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Arguments: 100, [c_last_name#35 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] +(42) TakeOrderedAndProject +Input [8]: [c_last_name#37, c_first_name#36, ca_city#39, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] +Arguments: 100, [c_last_name#37 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#39, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(40) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#38, d_dom#39] +(43) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#11, d_year#12, d_dom#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [3]: [d_date_sk#11, d_year#38, d_dom#39] -Condition : ((((isnotnull(d_dom#39) AND (d_dom#39 >= 1)) AND (d_dom#39 <= 2)) AND d_year#38 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) +(44) CometFilter +Input [3]: [d_date_sk#11, d_year#12, d_dom#13] +Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) -(42) CometProject -Input [3]: [d_date_sk#11, d_year#38, d_dom#39] +(45) CometProject +Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(43) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(44) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt index f2680bebb0..1ecde47e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - WholeStageCodegen (8) + WholeStageCodegen (4) Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] @@ -7,18 +7,18 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] InputAdapter Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,37 +29,32 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ CometProject [d_date_sk] CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange #4 + CometProject [s_store_sk] + CometFilter [s_city,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometBroadcastExchange #5 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #6 + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 + BroadcastExchange #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt index d56b01d737..b931de2923 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (25) - : : +- * BroadcastHashJoin LeftAnti BuildRight (24) - : : :- * BroadcastHashJoin LeftAnti BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.customer_address (26) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (27) + : : +- * BroadcastHashJoin LeftAnti BuildRight (26) + : : :- * BroadcastHashJoin LeftAnti BuildRight (19) + : : : :- * ColumnarToRow (12) + : : : : +- CometBroadcastHashJoin (11) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (10) + : : : : +- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometProject (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer_address (28) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) Scan parquet spark_catalog.default.customer @@ -54,228 +56,237 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#7] +(5) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(6) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(7) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] +(9) CometProject Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#4] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ss_customer_sk#4] +Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight + +(12) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(13) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(13) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#11] +(15) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(16) CometProject +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] +(17) ColumnarToRow [codegen id : 1] +Input [1]: [ws_bill_customer_sk#10] -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) BroadcastExchange +Input [1]: [ws_bill_customer_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 9] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#8] +Right keys [1]: [ws_bill_customer_sk#10] Join type: LeftAnti Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#17] -(20) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#15] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +Right output [1]: [d_date_sk#17] +Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(23) CometProject +Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] +Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#12] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15] +(24) ColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#14] -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#12] +Right keys [1]: [cs_ship_customer_sk#14] Join type: LeftAnti Join condition: None -(25) Project [codegen id : 9] +(27) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(26) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] +(28) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [GA,KY,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(27) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : (ca_state#17 IN (KY,GA,NM) AND isnotnull(ca_address_sk#16)) +(29) CometFilter +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) -(28) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] +(30) CometProject +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: [ca_address_sk#18], [ca_address_sk#18] -(29) ColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#16] +(31) ColumnarToRow [codegen id : 3] +Input [1]: [ca_address_sk#18] -(30) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(32) BroadcastExchange +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(31) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#16] +Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 9] +(34) Project [codegen id : 5] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] -(33) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +(35) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(34) CometFilter -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Condition : isnotnull(cd_demo_sk#18) +(36) CometFilter +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Condition : isnotnull(cd_demo_sk#20) -(35) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +(37) ColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -(36) BroadcastExchange -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(38) BroadcastExchange +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] +Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(38) Project [codegen id : 9] -Output [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +(40) Project [codegen id : 5] +Output [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -(39) HashAggregate [codegen id : 9] -Input [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Keys [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +(41) HashAggregate [codegen id : 5] +Input [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Keys [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#24] -Results [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] +Aggregate Attributes [1]: [count#26] +Results [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#27] -(40) Exchange -Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] -Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(42) Exchange +Input [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#27] +Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(41) HashAggregate [codegen id : 10] -Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] -Keys [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +(43) HashAggregate [codegen id : 6] +Input [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#27] +Keys [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#26] -Results [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, count(1)#26 AS cnt1#27, cd_purchase_estimate#22, count(1)#26 AS cnt2#28, cd_credit_rating#23, count(1)#26 AS cnt3#29] +Aggregate Attributes [1]: [count(1)#28] +Results [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#28 AS cnt1#29, cd_purchase_estimate#24, count(1)#28 AS cnt2#30, cd_credit_rating#25, count(1)#28 AS cnt3#31] -(42) TakeOrderedAndProject -Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] -Arguments: 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] +(44) TakeOrderedAndProject +Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#29, cd_purchase_estimate#24, cnt2#30, cd_credit_rating#25, cnt3#31] +Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#29, cd_purchase_estimate#24, cnt2#30, cd_credit_rating#25, cnt3#31] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(43) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#30, d_moy#31] +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter -Input [3]: [d_date_sk#7, d_year#30, d_moy#31] -Condition : (((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2001)) AND (d_moy#31 >= 4)) AND (d_moy#31 <= 6)) AND isnotnull(d_date_sk#7)) +(46) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) -(45) CometProject -Input [3]: [d_date_sk#7, d_year#30, d_moy#31] +(47) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt index f5b4eccfbe..de2d5eeda6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (10) + WholeStageCodegen (6) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] InputAdapter Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (9) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -12,62 +12,58 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [ca_address_sk] CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt index 790a917e38..5755ed1dc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt @@ -3,29 +3,29 @@ TakeOrderedAndProject (30) +- * HashAggregate (29) +- Exchange (28) +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) + +- * ColumnarToRow (26) + +- CometProject (25) + +- CometBroadcastHashJoin (24) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.date_dim (9) + : +- CometBroadcastExchange (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.item (15) + +- CometBroadcastExchange (23) +- CometProject (22) +- CometFilter (21) +- CometScan parquet spark_catalog.default.promotion (20) @@ -43,135 +43,138 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) ColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#10] -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [cd_demo_sk#10] +Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +(8) CometProject Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] + +(9) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +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 -(11) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#14] +(10) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(11) CometProject +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14], [d_date_sk#14] -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(14) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] +(15) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#16, i_item_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : isnotnull(i_item_sk#15) - -(16) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#15, i_item_id#16] +(16) CometFilter +Input [2]: [i_item_sk#16, i_item_id#17] +Condition : isnotnull(i_item_sk#16) -(17) BroadcastExchange -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [2]: [i_item_sk#16, i_item_id#17] +Arguments: [i_item_sk#16, i_item_id#17] -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [2]: [i_item_sk#16, i_item_id#17] +Arguments: [ss_item_sk#1], [i_item_sk#16], Inner, BuildRight -(19) Project [codegen id : 5] -Output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#15, i_item_id#16] +(19) CometProject +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#16, i_item_id#17] +Arguments: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17], [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] (20) Scan parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] +Output [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [Or(EqualTo(p_channel_email,N),EqualTo(p_channel_event,N)), IsNotNull(p_promo_sk)] ReadSchema: struct (21) CometFilter -Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] -Condition : (((p_channel_email#18 = N) OR (p_channel_event#19 = N)) AND isnotnull(p_promo_sk#17)) +Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] +Condition : (((p_channel_email#19 = N) OR (p_channel_event#20 = N)) AND isnotnull(p_promo_sk#18)) (22) CometProject -Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] -Arguments: [p_promo_sk#17], [p_promo_sk#17] +Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] +Arguments: [p_promo_sk#18], [p_promo_sk#18] -(23) ColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#17] +(23) CometBroadcastExchange +Input [1]: [p_promo_sk#18] +Arguments: [p_promo_sk#18] -(24) BroadcastExchange -Input [1]: [p_promo_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(24) CometBroadcastHashJoin +Left output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] +Right output [1]: [p_promo_sk#18] +Arguments: [ss_promo_sk#3], [p_promo_sk#18], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#17] -Join type: Inner -Join condition: None +(25) CometProject +Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17, p_promo_sk#18] +Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -(26) Project [codegen id : 5] -Output [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16] -Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16, p_promo_sk#17] +(26) ColumnarToRow [codegen id : 1] +Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -(27) HashAggregate [codegen id : 5] -Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16] -Keys [1]: [i_item_id#16] +(27) HashAggregate [codegen id : 1] +Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] +Keys [1]: [i_item_id#17] Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [8]: [sum#20, count#21, sum#22, count#23, sum#24, count#25, sum#26, count#27] -Results [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Aggregate Attributes [8]: [sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] +Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] (28) Exchange -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] +Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [1]: [i_item_id#16] +(29) HashAggregate [codegen id : 2] +Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] +Keys [1]: [i_item_id#17] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#36, avg(UnscaledValue(ss_list_price#5))#37, avg(UnscaledValue(ss_coupon_amt#7))#38, avg(UnscaledValue(ss_sales_price#6))#39] -Results [5]: [i_item_id#16, avg(ss_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(ss_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(ss_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(ss_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] +Aggregate Attributes [4]: [avg(ss_quantity#4)#37, avg(UnscaledValue(ss_list_price#5))#38, avg(UnscaledValue(ss_coupon_amt#7))#39, avg(UnscaledValue(ss_sales_price#6))#40] +Results [5]: [i_item_id#17, avg(ss_quantity#4)#37 AS agg1#41, cast((avg(UnscaledValue(ss_list_price#5))#38 / 100.0) as decimal(11,6)) AS agg2#42, cast((avg(UnscaledValue(ss_coupon_amt#7))#39 / 100.0) as decimal(11,6)) AS agg3#43, cast((avg(UnscaledValue(ss_sales_price#6))#40 / 100.0) as decimal(11,6)) AS agg4#44] (30) TakeOrderedAndProject -Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] -Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] +Input [5]: [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] +Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] ===== Subqueries ===== @@ -184,18 +187,18 @@ BroadcastExchange (35) (31) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] 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 (32) CometFilter -Input [2]: [d_date_sk#14, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) (33) CometProject -Input [2]: [d_date_sk#14, d_year#44] +Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] (34) ColumnarToRow [codegen id : 1] @@ -203,6 +206,6 @@ Input [1]: [d_date_sk#14] (35) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt index 2471de20a3..0429448f51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -25,28 +25,18 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometBroadcastExchange #3 + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #5 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #6 + CometProject [p_promo_sk] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt index c3acf0a8de..0a69349606 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt @@ -1,47 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * Sort (40) - +- Exchange (39) - +- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Expand (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (32) - +- * BroadcastHashJoin LeftSemi BuildRight (31) - :- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (30) - +- * Project (29) - +- * Filter (28) - +- Window (27) - +- WindowGroupLimit (26) - +- * Sort (25) - +- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.store (13) - +- ReusedExchange (19) +TakeOrderedAndProject (45) ++- * Project (44) + +- Window (43) + +- * Sort (42) + +- Exchange (41) + +- * HashAggregate (40) + +- Exchange (39) + +- * HashAggregate (38) + +- * Expand (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * ColumnarToRow (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.date_dim (3) + +- BroadcastExchange (34) + +- * BroadcastHashJoin LeftSemi BuildRight (33) + :- * ColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.store (10) + +- BroadcastExchange (32) + +- * Project (31) + +- * Filter (30) + +- Window (29) + +- WindowGroupLimit (28) + +- * Sort (27) + +- * HashAggregate (26) + +- Exchange (25) + +- * HashAggregate (24) + +- * ColumnarToRow (23) + +- CometProject (22) + +- CometBroadcastHashJoin (21) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store_sales (13) + : +- CometBroadcastExchange (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.store (15) + +- ReusedExchange (20) (1) Scan parquet spark_catalog.default.store_sales @@ -56,228 +58,238 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] +(7) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] + +(9) ColumnarToRow [codegen id : 5] +Input [2]: [ss_store_sk#1, ss_net_profit#2] -(7) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] +(10) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) +(11) CometFilter +Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Condition : isnotnull(s_store_sk#7) -(9) ColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] +(12) ColumnarToRow [codegen id : 4] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(13) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) CometFilter -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(12) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(14) CometFilter +Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_store_sk#10) -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] +(15) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : isnotnull(s_store_sk#13) +(16) CometFilter +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) -(15) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#13, s_state#14] +(17) CometBroadcastExchange +Input [2]: [s_store_sk#14, s_state#15] +Arguments: [s_store_sk#14, s_state#15] -(16) BroadcastExchange -Input [2]: [s_store_sk#13, s_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(18) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Right output [2]: [s_store_sk#14, s_state#15] +Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None +(19) CometProject +Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#15] +Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15], [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] -(18) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] -(19) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#15] +(21) CometBroadcastHashJoin +Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] +Right output [1]: [d_date_sk#16] +Arguments: [ss_sold_date_sk#12], [d_date_sk#16], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(22) CometProject +Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15, d_date_sk#16] +Arguments: [ss_net_profit#11, s_state#15], [ss_net_profit#11, s_state#15] + +(23) ColumnarToRow [codegen id : 1] +Input [2]: [ss_net_profit#11, s_state#15] + +(24) HashAggregate [codegen id : 1] +Input [2]: [ss_net_profit#11, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] +Aggregate Attributes [1]: [sum#17] +Results [2]: [s_state#15, sum#18] + +(25) Exchange +Input [2]: [s_state#15, sum#18] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(26) HashAggregate [codegen id : 2] +Input [2]: [s_state#15, sum#18] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#19] +Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#19,17,2) AS _w0#20, s_state#15] + +(27) Sort [codegen id : 2] +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [s_state#15 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 -(21) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#14] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] - -(22) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#14] -Keys [1]: [s_state#14] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [s_state#14, sum#17] - -(23) Exchange -Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(24) HashAggregate [codegen id : 5] -Input [2]: [s_state#14, sum#17] -Keys [1]: [s_state#14] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] -Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] - -(25) Sort [codegen id : 5] -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 - -(26) WindowGroupLimit -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(27) Window -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] - -(28) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] -Condition : (ranking#20 <= 5) - -(29) Project [codegen id : 6] -Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] - -(30) BroadcastExchange -Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(31) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [s_state#8] -Right keys [1]: [s_state#14] +(28) WindowGroupLimit +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [s_state#15], [_w0#20 DESC NULLS LAST], rank(_w0#20), 5, Final + +(29) Window +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [rank(_w0#20) windowspecdefinition(s_state#15, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#15], [_w0#20 DESC NULLS LAST] + +(30) Filter [codegen id : 3] +Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] +Condition : (ranking#21 <= 5) + +(31) Project [codegen id : 3] +Output [1]: [s_state#15] +Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] + +(32) BroadcastExchange +Input [1]: [s_state#15] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] + +(33) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [s_state#9] +Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(32) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(34) BroadcastExchange +Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] +Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(34) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_state#8, s_county#7] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] +(36) Project [codegen id : 5] +Output [3]: [ss_net_profit#2, s_state#9, s_county#8] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#9] -(35) Expand [codegen id : 8] -Input [3]: [ss_net_profit#2, s_state#8, s_county#7] -Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] +(37) Expand [codegen id : 5] +Input [3]: [ss_net_profit#2, s_state#9, s_county#8] +Arguments: [[ss_net_profit#2, s_state#9, s_county#8, 0], [ss_net_profit#2, s_state#9, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -(36) HashAggregate [codegen id : 8] -Input [4]: [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] -Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] +(38) HashAggregate [codegen id : 5] +Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#24] -Results [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] +Aggregate Attributes [1]: [sum#25] +Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -(37) Exchange -Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] -Arguments: hashpartitioning(s_state#21, s_county#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(39) Exchange +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(38) HashAggregate [codegen id : 9] -Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] -Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] +(40) HashAggregate [codegen id : 6] +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#26] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS total_sum#27, s_state#21, s_county#22, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS lochierarchy#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS _w0#29, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS _w1#30, CASE WHEN (cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint) = 0) THEN s_state#21 END AS _w2#31] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#28, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#29, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS _w0#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32] -(39) Exchange -Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] -Arguments: hashpartitioning(_w1#30, _w2#31, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(41) Exchange +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] +Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(40) Sort [codegen id : 10] -Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] -Arguments: [_w1#30 ASC NULLS FIRST, _w2#31 ASC NULLS FIRST, _w0#29 DESC NULLS LAST], false, 0 +(42) Sort [codegen id : 7] +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] +Arguments: [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST], false, 0 -(41) Window -Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] -Arguments: [rank(_w0#29) windowspecdefinition(_w1#30, _w2#31, _w0#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#32], [_w1#30, _w2#31], [_w0#29 DESC NULLS LAST] +(43) Window +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] +Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] -(42) Project [codegen id : 11] -Output [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] -Input [8]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31, rank_within_parent#32] +(44) Project [codegen id : 8] +Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] +Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] -(43) TakeOrderedAndProject -Input [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] -Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN s_state#21 END ASC NULLS FIRST, rank_within_parent#32 ASC NULLS FIRST], [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] +(45) TakeOrderedAndProject +Input [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] +Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#33 ASC NULLS FIRST], [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(44) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#33] +(46) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#33] -Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1200)) AND (d_month_seq#33 <= 1211)) AND isnotnull(d_date_sk#5)) +(47) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(46) CometProject -Input [2]: [d_date_sk#5, d_month_seq#33] +(48) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(47) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(48) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt index 268af2e2f3..c962534aad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (11) + WholeStageCodegen (8) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (10) + WholeStageCodegen (7) Sort [_w1,_w2,_w0] InputAdapter Exchange [_w1,_w2] #1 - WholeStageCodegen (9) + WholeStageCodegen (6) HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] InputAdapter Exchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] Expand [ss_net_profit,s_state,s_county] Project [ss_net_profit,s_state,s_county] BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -30,46 +30,43 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) + BroadcastExchange #5 + WholeStageCodegen (4) BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) + BroadcastExchange #6 + WholeStageCodegen (3) Project [s_state] Filter [ranking] InputAdapter Window [_w0,s_state] WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) + WholeStageCodegen (2) Sort [s_state,_w0] HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] InputAdapter - Exchange [s_state] #6 - WholeStageCodegen (4) + Exchange [s_state] #7 + WholeStageCodegen (1) HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #8 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt index 624103b664..8ec2c5c5a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt @@ -1,42 +1,41 @@ == Physical Plan == -* Sort (38) -+- Exchange (37) - +- * HashAggregate (36) - +- Exchange (35) - +- * HashAggregate (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildLeft (25) - : :- BroadcastExchange (5) - : : +- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.item (1) - : +- Union (24) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.web_sales (6) - : : +- ReusedExchange (9) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * ColumnarToRow (14) - : : : +- CometFilter (13) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : +- ReusedExchange (15) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * ColumnarToRow (20) - : : +- CometFilter (19) - : : +- CometScan parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (21) - +- BroadcastExchange (31) - +- * ColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.time_dim (27) +* Sort (37) ++- Exchange (36) + +- * HashAggregate (35) + +- Exchange (34) + +- * HashAggregate (33) + +- * ColumnarToRow (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometBroadcastExchange (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.item (1) + : +- CometUnion (23) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.web_sales (5) + : : +- CometBroadcastExchange (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.date_dim (7) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : +- ReusedExchange (15) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (19) + : : +- CometScan parquet spark_catalog.default.store_sales (18) + : +- ReusedExchange (20) + +- CometBroadcastExchange (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.time_dim (26) (1) Scan parquet spark_catalog.default.item @@ -54,14 +53,11 @@ Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull( Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] Arguments: [i_item_sk#1, i_brand_id#2, i_brand#3], [i_item_sk#1, i_brand_id#2, i_brand#3] -(4) ColumnarToRow [codegen id : 1] +(4) CometBroadcastExchange Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#3] +Arguments: [i_item_sk#1, i_brand_id#2, i_brand#3] -(5) BroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#3] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(6) Scan parquet spark_catalog.default.web_sales +(5) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -69,186 +65,191 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#8), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] Condition : (isnotnull(ws_item_sk#6) AND isnotnull(ws_sold_time_sk#5)) -(8) ColumnarToRow [codegen id : 3] -Input [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] +(7) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] +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 + +(8) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11)) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) + +(9) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Arguments: [d_date_sk#10], [d_date_sk#10] -(9) ReusedExchange [Reuses operator id: 43] -Output [1]: [d_date_sk#10] +(10) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [ws_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight -(11) Project [codegen id : 3] -Output [3]: [ws_ext_sales_price#7 AS ext_price#11, ws_item_sk#6 AS sold_item_sk#12, ws_sold_time_sk#5 AS time_sk#13] +(12) CometProject Input [5]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8, d_date_sk#10] +Arguments: [ext_price#13, sold_item_sk#14, time_sk#15], [ws_ext_sales_price#7 AS ext_price#13, ws_item_sk#6 AS sold_item_sk#14, ws_sold_time_sk#5 AS time_sk#15] -(12) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +(13) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_sold_time_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] ReadSchema: struct -(13) CometFilter -Input [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_item_sk#15) AND isnotnull(cs_sold_time_sk#14)) - -(14) ColumnarToRow [codegen id : 5] -Input [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +(14) CometFilter +Input [4]: [cs_sold_time_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_sold_time_sk#16)) -(15) ReusedExchange [Reuses operator id: 43] -Output [1]: [d_date_sk#19] +(15) ReusedExchange [Reuses operator id: 10] +Output [1]: [d_date_sk#21] -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [4]: [cs_sold_time_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Right output [1]: [d_date_sk#21] +Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight -(17) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#16 AS ext_price#20, cs_item_sk#15 AS sold_item_sk#21, cs_sold_time_sk#14 AS time_sk#22] -Input [5]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17, d_date_sk#19] +(17) CometProject +Input [5]: [cs_sold_time_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] +Arguments: [ext_price#22, sold_item_sk#23, time_sk#24], [cs_ext_sales_price#18 AS ext_price#22, cs_item_sk#17 AS sold_item_sk#23, cs_sold_time_sk#16 AS time_sk#24] (18) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct (19) CometFilter -Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) +Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_time_sk#25)) -(20) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +(20) ReusedExchange [Reuses operator id: 10] +Output [1]: [d_date_sk#30] -(21) ReusedExchange [Reuses operator id: 43] -Output [1]: [d_date_sk#28] +(21) CometBroadcastHashJoin +Left output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Right output [1]: [d_date_sk#30] +Arguments: [ss_sold_date_sk#28], [d_date_sk#30], Inner, BuildRight -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#28] -Join type: Inner -Join condition: None +(22) CometProject +Input [5]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30] +Arguments: [ext_price#31, sold_item_sk#32, time_sk#33], [ss_ext_sales_price#27 AS ext_price#31, ss_item_sk#26 AS sold_item_sk#32, ss_sold_time_sk#25 AS time_sk#33] -(23) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#25 AS ext_price#29, ss_item_sk#24 AS sold_item_sk#30, ss_sold_time_sk#23 AS time_sk#31] -Input [5]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#28] +(23) CometUnion +Child 0 Input [3]: [ext_price#13, sold_item_sk#14, time_sk#15] +Child 1 Input [3]: [ext_price#22, sold_item_sk#23, time_sk#24] +Child 2 Input [3]: [ext_price#31, sold_item_sk#32, time_sk#33] -(24) Union +(24) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand_id#2, i_brand#3] +Right output [3]: [ext_price#13, sold_item_sk#14, time_sk#15] +Arguments: [i_item_sk#1], [sold_item_sk#14], Inner, BuildLeft -(25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [sold_item_sk#12] -Join type: Inner -Join condition: None +(25) CometProject +Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#13, sold_item_sk#14, time_sk#15] +Arguments: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15], [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15] -(26) Project [codegen id : 9] -Output [4]: [i_brand_id#2, i_brand#3, ext_price#11, time_sk#13] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#11, sold_item_sk#12, time_sk#13] - -(27) Scan parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +(26) Scan parquet spark_catalog.default.time_dim +Output [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) - -(29) CometProject -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Arguments: [t_time_sk#32, t_hour#33, t_minute#34], [t_time_sk#32, t_hour#33, t_minute#34] - -(30) ColumnarToRow [codegen id : 8] -Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] - -(31) BroadcastExchange -Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [time_sk#13] -Right keys [1]: [t_time_sk#32] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#3, ext_price#11, t_hour#33, t_minute#34] -Input [7]: [i_brand_id#2, i_brand#3, ext_price#11, time_sk#13, t_time_sk#32, t_hour#33, t_minute#34] - -(34) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#3, ext_price#11, t_hour#33, t_minute#34] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] -Functions [1]: [partial_sum(UnscaledValue(ext_price#11))] -Aggregate Attributes [1]: [sum#36] -Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] - -(35) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(36) HashAggregate [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] -Functions [1]: [sum(UnscaledValue(ext_price#11))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#11))#38] -Results [5]: [i_brand_id#2 AS brand_id#39, i_brand#3 AS brand#40, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#11))#38,17,2) AS ext_price#41] - -(37) Exchange -Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] -Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(38) Sort [codegen id : 11] -Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] -Arguments: [ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST], true, 0 +(27) CometFilter +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Condition : (((t_meal_time#37 = breakfast ) OR (t_meal_time#37 = dinner )) AND isnotnull(t_time_sk#34)) + +(28) CometProject +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Arguments: [t_time_sk#34, t_hour#35, t_minute#36], [t_time_sk#34, t_hour#35, t_minute#36] + +(29) CometBroadcastExchange +Input [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: [t_time_sk#34, t_hour#35, t_minute#36] + +(30) CometBroadcastHashJoin +Left output [4]: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15] +Right output [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: [time_sk#15], [t_time_sk#34], Inner, BuildRight + +(31) CometProject +Input [7]: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15, t_time_sk#34, t_hour#35, t_minute#36] +Arguments: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36], [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36] + +(32) ColumnarToRow [codegen id : 1] +Input [5]: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36] + +(33) HashAggregate [codegen id : 1] +Input [5]: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] +Functions [1]: [partial_sum(UnscaledValue(ext_price#13))] +Aggregate Attributes [1]: [sum#38] +Results [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#39] + +(34) Exchange +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#39] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(35) HashAggregate [codegen id : 2] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#39] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] +Functions [1]: [sum(UnscaledValue(ext_price#13))] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#13))#40] +Results [5]: [i_brand_id#2 AS brand_id#41, i_brand#3 AS brand#42, t_hour#35, t_minute#36, MakeDecimal(sum(UnscaledValue(ext_price#13))#40,17,2) AS ext_price#43] + +(36) Exchange +Input [5]: [brand_id#41, brand#42, t_hour#35, t_minute#36, ext_price#43] +Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(37) Sort [codegen id : 3] +Input [5]: [brand_id#41, brand#42, t_hour#35, t_minute#36, ext_price#43] +Arguments: [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (43) -+- * ColumnarToRow (42) - +- CometProject (41) - +- CometFilter (40) - +- CometScan parquet spark_catalog.default.date_dim (39) +Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (42) ++- * ColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.date_dim (38) -(39) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#42, d_moy#43] +(38) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] 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 -(40) CometFilter -Input [3]: [d_date_sk#10, d_year#42, d_moy#43] -Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 11)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#10)) +(39) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11)) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) -(41) CometProject -Input [3]: [d_date_sk#10, d_year#42, d_moy#43] +(40) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(42) ColumnarToRow [codegen id : 1] +(41) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(43) BroadcastExchange +(42) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt index bea5376a00..c732d903fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt @@ -1,69 +1,53 @@ -WholeStageCodegen (11) +WholeStageCodegen (3) Sort [ext_price,brand_id] InputAdapter Exchange [ext_price,brand_id] #1 - WholeStageCodegen (10) + WholeStageCodegen (2) HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id,t_hour,t_minute] #2 - WholeStageCodegen (9) + WholeStageCodegen (1) HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] - BroadcastHashJoin [time_sk,t_time_sk] - Project [i_brand_id,i_brand,ext_price,time_sk] - BroadcastHashJoin [i_item_sk,sold_item_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - InputAdapter - Union - WholeStageCodegen (3) - Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_sold_time_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (5) - Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_sold_time_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (7) - Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_sold_time_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + ColumnarToRow + InputAdapter + CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] + CometBroadcastHashJoin [time_sk,t_time_sk] + CometProject [i_brand_id,i_brand,ext_price,time_sk] + CometBroadcastHashJoin [i_item_sk,sold_item_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometUnion + CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_sold_time_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_sold_time_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_sold_time_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #6 + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_meal_time,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt index 4e5d9e9f60..26edd145c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt @@ -1,74 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * SortMergeJoin LeftOuter (65) - :- * Sort (58) - : +- Exchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * ColumnarToRow (3) - : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * ColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * ColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * ColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * ColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * ColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * ColumnarToRow (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.promotion (51) - +- * Sort (64) - +- Exchange (63) - +- * ColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.catalog_returns (59) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- * Project (63) + +- * SortMergeJoin LeftOuter (62) + :- * Sort (55) + : +- Exchange (54) + : +- * Project (53) + : +- * BroadcastHashJoin LeftOuter BuildRight (52) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * ColumnarToRow (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (35) + : : : : +- CometBroadcastHashJoin (34) + : : : : :- CometProject (29) + : : : : : +- CometBroadcastHashJoin (28) + : : : : : :- CometProject (23) + : : : : : : +- CometBroadcastHashJoin (22) + : : : : : : :- CometProject (17) + : : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : : :- CometProject (12) + : : : : : : : : +- CometBroadcastHashJoin (11) + : : : : : : : : :- CometProject (7) + : : : : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : : : : :- CometFilter (2) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- CometBroadcastExchange (5) + : : : : : : : : : +- CometFilter (4) + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (3) + : : : : : : : : +- CometBroadcastExchange (10) + : : : : : : : : +- CometFilter (9) + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (8) + : : : : : : : +- CometBroadcastExchange (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : : +- CometBroadcastExchange (21) + : : : : : : +- CometProject (20) + : : : : : : +- CometFilter (19) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) + : : : : : +- CometBroadcastExchange (27) + : : : : : +- CometProject (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan parquet spark_catalog.default.household_demographics (24) + : : : : +- CometBroadcastExchange (33) + : : : : +- CometProject (32) + : : : : +- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.date_dim (30) + : : : +- CometBroadcastExchange (38) + : : : +- CometFilter (37) + : : : +- CometScan parquet spark_catalog.default.date_dim (36) + : : +- BroadcastExchange (45) + : : +- * ColumnarToRow (44) + : : +- CometFilter (43) + : : +- CometScan parquet spark_catalog.default.date_dim (42) + : +- BroadcastExchange (51) + : +- * ColumnarToRow (50) + : +- CometFilter (49) + : +- CometScan parquet spark_catalog.default.promotion (48) + +- * Sort (61) + +- Exchange (60) + +- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.catalog_returns (56) (1) Scan parquet spark_catalog.default.catalog_sales @@ -83,10 +80,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -(7) BroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) +(6) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] +Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +(7) CometProject Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -(10) Scan parquet spark_catalog.default.warehouse +(8) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [w_warehouse_sk#14, w_warehouse_name#15] -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +(12) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#16, i_item_desc#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#16, i_item_desc#17] - -(19) BroadcastExchange +(15) CometBroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#16, i_item_desc#17] -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +Right output [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(17) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(22) Scan parquet spark_catalog.default.customer_demographics +(18) Scan parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#18, cd_marital_status#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_marital_status), EqualTo(cd_marital_status,D), IsNotNull(cd_demo_sk)] ReadSchema: struct -(23) CometFilter +(19) CometFilter Input [2]: [cd_demo_sk#18, cd_marital_status#19] Condition : ((isnotnull(cd_marital_status#19) AND (cd_marital_status#19 = D)) AND isnotnull(cd_demo_sk#18)) -(24) CometProject +(20) CometProject Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) ColumnarToRow [codegen id : 4] +(21) CometBroadcastExchange Input [1]: [cd_demo_sk#18] +Arguments: [cd_demo_sk#18] -(26) BroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [cd_demo_sk#18] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(23) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] +Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(29) Scan parquet spark_catalog.default.household_demographics +(24) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#20, hd_buy_potential#21] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct -(30) CometFilter +(25) CometFilter Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Condition : ((isnotnull(hd_buy_potential#21) AND (hd_buy_potential#21 = >10000 )) AND isnotnull(hd_demo_sk#20)) -(31) CometProject +(26) CometProject Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) ColumnarToRow [codegen id : 5] +(27) CometBroadcastExchange Input [1]: [hd_demo_sk#20] +Arguments: [hd_demo_sk#20] -(33) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(28) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [hd_demo_sk#20] +Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(29) CometProject Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +(30) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +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 -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(31) CometFilter +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)) -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +(32) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] + +(33) CometBroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] + +(34) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight + +(35) CometProject Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(39) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] +(36) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_week_seq#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) +(37) CometFilter +Input [2]: [d_date_sk#26, d_week_seq#27] +Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) -(41) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] +(38) CometBroadcastExchange +Input [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_date_sk#26, d_week_seq#27] -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] +(39) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +Right output [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None +(40) CometProject +Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] +(41) ColumnarToRow [codegen id : 3] +Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(45) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] +(42) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_date#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) +(43) CometFilter +Input [2]: [d_date_sk#28, d_date#29] +Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) -(47) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] +(44) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_date#29] -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(45) BroadcastExchange +Input [2]: [d_date_sk#28, d_date#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(49) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#28] Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) +Join condition: (d_date#29 > date_add(d_date#23, 5)) -(50) Project [codegen id : 10] +(47) Project [codegen id : 3] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] +Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -(51) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] +(48) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) +(49) CometFilter +Input [1]: [p_promo_sk#30] +Condition : isnotnull(p_promo_sk#30) -(53) ColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] +(50) ColumnarToRow [codegen id : 2] +Input [1]: [p_promo_sk#30] -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(51) BroadcastExchange +Input [1]: [p_promo_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(55) BroadcastHashJoin [codegen id : 10] +(52) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] +Right keys [1]: [p_promo_sk#30] Join type: LeftOuter Join condition: None -(56) Project [codegen id : 10] +(53) Project [codegen id : 3] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -(57) Exchange +(54) Exchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(58) Sort [codegen id : 11] +(55) Sort [codegen id : 4] Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 -(59) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +(56) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) +(57) CometFilter +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] +(58) CometProject +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] -(62) ColumnarToRow [codegen id : 12] -Input [2]: [cr_item_sk#30, cr_order_number#31] +(59) ColumnarToRow [codegen id : 5] +Input [2]: [cr_item_sk#31, cr_order_number#32] -(63) Exchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(60) Exchange +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(64) Sort [codegen id : 13] -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST], false, 0 +(61) Sort [codegen id : 6] +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 14] +(62) SortMergeJoin [codegen id : 7] Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#30, cr_order_number#31] +Right keys [2]: [cr_item_sk#31, cr_order_number#32] Join type: LeftOuter Join condition: None -(66) Project [codegen id : 14] +(63) Project [codegen id : 7] Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -(67) HashAggregate [codegen id : 14] +(64) HashAggregate [codegen id : 7] Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Aggregate Attributes [1]: [count#34] +Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] -(68) Exchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(65) Exchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(69) HashAggregate [codegen id : 15] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +(66) HashAggregate [codegen id : 8] +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#35 AS no_promo#36, count(1)#35 AS promo#37, count(1)#35 AS total_cnt#38] +Aggregate Attributes [1]: [count(1)#36] +Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#36 AS no_promo#37, count(1)#36 AS promo#38, count(1)#36 AS total_cnt#39] -(70) TakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] -Arguments: 100, [total_cnt#38 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] +(67) TakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#37, promo#38, total_cnt#39] +Arguments: 100, [total_cnt#39 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#37, promo#38, total_cnt#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(71) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +(68) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] 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 -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] -Condition : ((((isnotnull(d_year#39) AND (d_year#39 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(69) CometFilter +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)) -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +(70) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(75) BroadcastExchange +(72) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt index 17fc9dee7e..bea1fd4a16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt @@ -1,38 +1,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - WholeStageCodegen (15) + WholeStageCodegen (8) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - WholeStageCodegen (14) + WholeStageCodegen (7) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] Project [w_warehouse_name,i_item_desc,d_week_seq] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (4) Sort [cs_item_sk,cs_order_number] InputAdapter Exchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) + WholeStageCodegen (3) 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] BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -43,72 +43,50 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_year,d_date_sk,d_week_seq,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [d_week_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange #4 + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange #7 + CometProject [cd_demo_sk] + CometFilter [cd_marital_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange #8 + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange #10 + CometFilter [d_week_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) + BroadcastExchange #11 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) + BroadcastExchange #12 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (6) Sort [cr_item_sk,cr_order_number] InputAdapter - Exchange [cr_item_sk,cr_order_number] #12 - WholeStageCodegen (12) + Exchange [cr_item_sk,cr_order_number] #13 + WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt index ef2c38aea4..88dcba0fb4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt @@ -1,36 +1,37 @@ == Physical Plan == -* Sort (32) -+- Exchange (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (24) - : +- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (28) - +- * ColumnarToRow (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.customer (25) +* Sort (33) ++- Exchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Filter (25) + : +- * HashAggregate (24) + : +- Exchange (23) + : +- * ColumnarToRow (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.household_demographics (15) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.customer (26) (1) Scan parquet spark_catalog.default.store_sales @@ -45,174 +46,179 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#7] +(4) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(7) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +(8) CometProject Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#10, s_county#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : (s_county#9 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] +(10) CometFilter +Input [2]: [s_store_sk#10, s_county#11] +Condition : (s_county#11 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#10)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] +(11) CometProject +Input [2]: [s_store_sk#10, s_county#11] +Arguments: [s_store_sk#10], [s_store_sk#10] -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#10] +Arguments: [s_store_sk#10] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +Right output [1]: [s_store_sk#10] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] +(14) CometProject +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -(14) Scan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +(15) Scan parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.0) END) AND isnotnull(hd_demo_sk#10)) +(16) CometFilter +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Condition : ((((isnotnull(hd_vehicle_count#15) AND ((hd_buy_potential#13 = >10000 ) OR (hd_buy_potential#13 = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN ((cast(hd_dep_count#14 as double) / cast(hd_vehicle_count#15 as double)) > 1.0) END) AND isnotnull(hd_demo_sk#12)) -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] +(17) CometProject +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Arguments: [hd_demo_sk#12], [hd_demo_sk#12] -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#12] +Arguments: [hd_demo_sk#12] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(19) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Right output [1]: [hd_demo_sk#12] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None +(20) CometProject +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] +Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] +(21) CometHashAggregate Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) Exchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) ColumnarToRow [codegen id : 1] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] + +(23) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(23) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +(24) HashAggregate [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] +Aggregate Attributes [1]: [count(1)#17] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(24) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) +(25) Filter [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18] +Condition : ((cnt#18 >= 1) AND (cnt#18 <= 5)) -(25) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(26) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) +(27) CometFilter +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Condition : isnotnull(c_customer_sk#19) -(27) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(28) ColumnarToRow [codegen id : 2] +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(28) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(29) BroadcastExchange +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(29) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#19] Join type: Inner Join condition: None -(30) Project [codegen id : 6] -Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(31) Project [codegen id : 3] +Output [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18, c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(31) Exchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(32) Exchange +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: rangepartitioning(cnt#18 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 7] -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [cnt#17 DESC NULLS LAST], true, 0 +(33) Sort [codegen id : 4] +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: [cnt#18 DESC NULLS LAST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(33) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +(34) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_dom#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] -Condition : ((((isnotnull(d_dom#24) AND (d_dom#24 >= 1)) AND (d_dom#24 <= 2)) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(35) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(35) CometProject -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +(36) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(37) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt index 7c5ee1ef5a..060c3e1530 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt @@ -1,24 +1,24 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) Sort [cnt] InputAdapter Exchange [cnt] #1 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [cnt] HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter Exchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,27 +29,21 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange #5 + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange #6 + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt index dad94eb2cd..d5eb481f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt @@ -2,74 +2,74 @@ TakeOrderedAndProject (71) +- * Project (70) +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * ColumnarToRow (36) - : : : +- CometFilter (35) - : : : +- CometScan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (17) + : : : +- * HashAggregate (16) + : : : +- Exchange (15) + : : : +- * HashAggregate (14) + : : : +- * ColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- BroadcastExchange (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * HashAggregate (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- CometBroadcastExchange (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- BroadcastExchange (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * HashAggregate (47) + : +- * ColumnarToRow (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometProject (42) + : : +- CometBroadcastHashJoin (41) + : : :- CometFilter (37) + : : : +- CometScan parquet spark_catalog.default.customer (36) + : : +- CometBroadcastExchange (40) + : : +- CometFilter (39) + : : +- CometScan parquet spark_catalog.default.web_sales (38) + : +- ReusedExchange (43) +- BroadcastExchange (68) +- * HashAggregate (67) +- Exchange (66) +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * ColumnarToRow (55) - : : +- CometFilter (54) - : : +- CometScan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) + +- * ColumnarToRow (64) + +- CometProject (63) + +- CometBroadcastHashJoin (62) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (61) (1) Scan parquet spark_catalog.default.customer @@ -83,10 +83,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] Condition : isnotnull(ss_customer_sk#5) -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Right output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: [c_customer_sk#1], [ss_customer_sk#5], Inner, BuildRight -(9) Project [codegen id : 3] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +(7) CometProject Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 75] +(8) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#9, d_year#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#9, d_year#10] +Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [2]: [d_date_sk#9, d_year#10] +Arguments: [d_date_sk#9, d_year#10] -(12) Project [codegen id : 3] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] +(11) CometBroadcastHashJoin +Left output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#9, d_year#10] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight + +(12) CometProject Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] + +(13) ColumnarToRow [codegen id : 1] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] -(13) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] Aggregate Attributes [1]: [sum#11] Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -(14) Exchange +(15) Exchange Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 16] +(16) HashAggregate [codegen id : 8] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] -(16) Filter [codegen id : 16] +(17) Filter [codegen id : 8] Input [2]: [customer_id#14, year_total#15] Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) -(17) Scan parquet spark_catalog.default.customer +(18) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter +(19) CometFilter Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) -(19) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] - (20) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Batched: true @@ -176,79 +180,86 @@ ReadSchema: struct Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) -(22) ColumnarToRow [codegen id : 4] +(22) CometBroadcastExchange Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Arguments: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -(23) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +Right output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Arguments: [c_customer_sk#16], [ss_customer_sk#20], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#16] -Right keys [1]: [ss_customer_sk#20] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] +(24) CometProject Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Arguments: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22], [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] -(26) ReusedExchange [Reuses operator id: 79] +(25) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(26) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) + +(27) CometBroadcastExchange +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24, d_year#25] -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] +(28) CometBroadcastHashJoin +Left output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] +Right output [2]: [d_date_sk#24, d_year#25] +Arguments: [ss_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight + +(29) CometProject Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] +Arguments: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25], [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] + +(30) ColumnarToRow [codegen id : 2] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -(29) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] Aggregate Attributes [1]: [sum#26] Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -(30) Exchange +(32) Exchange Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(31) HashAggregate [codegen id : 7] +(33) HashAggregate [codegen id : 3] Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] -(32) BroadcastExchange +(34) BroadcastExchange Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 16] +(35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#28] Join type: Inner Join condition: None -(34) Scan parquet spark_catalog.default.customer +(36) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter +(37) CometFilter Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) -(36) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] - -(37) Scan parquet spark_catalog.default.web_sales +(38) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] @@ -256,90 +267,85 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(38) CometFilter +(39) CometFilter Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Condition : isnotnull(ws_bill_customer_sk#36) -(39) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] - -(40) BroadcastExchange +(40) CometBroadcastExchange Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#32] -Right keys [1]: [ws_bill_customer_sk#36] -Join type: Inner -Join condition: None +(41) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Right output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Arguments: [c_customer_sk#32], [ws_bill_customer_sk#36], Inner, BuildRight -(42) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] +(42) CometProject Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38], [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] -(43) ReusedExchange [Reuses operator id: 75] +(43) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#40, d_year#41] -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] +Right output [2]: [d_date_sk#40, d_year#41] +Arguments: [ws_sold_date_sk#38], [d_date_sk#40], Inner, BuildRight -(45) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] +(45) CometProject Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41], [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -(46) HashAggregate [codegen id : 10] +(46) ColumnarToRow [codegen id : 4] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] + +(47) HashAggregate [codegen id : 4] Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] Aggregate Attributes [1]: [sum#42] Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -(47) Exchange +(48) Exchange Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(48) HashAggregate [codegen id : 11] +(49) HashAggregate [codegen id : 5] Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] -(49) Filter [codegen id : 11] +(50) Filter [codegen id : 5] Input [2]: [customer_id#45, year_total#46] Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) -(50) BroadcastExchange +(51) BroadcastExchange Input [2]: [customer_id#45, year_total#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(51) BroadcastHashJoin [codegen id : 16] +(52) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#45] Join type: Inner Join condition: None -(52) Project [codegen id : 16] +(53) Project [codegen id : 8] Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] -(53) Scan parquet spark_catalog.default.customer +(54) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter +(55) CometFilter Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) -(55) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] - (56) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Batched: true @@ -352,37 +358,35 @@ ReadSchema: struct Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Condition : isnotnull(ws_bill_customer_sk#51) -(58) ColumnarToRow [codegen id : 12] +(58) CometBroadcastExchange Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Arguments: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -(59) BroadcastExchange -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(59) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +Right output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Arguments: [c_customer_sk#47], [ws_bill_customer_sk#51], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#47] -Right keys [1]: [ws_bill_customer_sk#51] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] +(60) CometProject Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Arguments: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53], [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] -(62) ReusedExchange [Reuses operator id: 79] +(61) ReusedExchange [Reuses operator id: 27] Output [2]: [d_date_sk#55, d_year#56] -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#53] -Right keys [1]: [d_date_sk#55] -Join type: Inner -Join condition: None +(62) CometBroadcastHashJoin +Left output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] +Right output [2]: [d_date_sk#55, d_year#56] +Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight -(64) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] +(63) CometProject Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] +Arguments: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56], [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -(65) HashAggregate [codegen id : 14] +(64) ColumnarToRow [codegen id : 6] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] + +(65) HashAggregate [codegen id : 6] Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] @@ -391,9 +395,9 @@ Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum# (66) Exchange Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(67) HashAggregate [codegen id : 15] +(67) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] @@ -402,15 +406,15 @@ Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue( (68) BroadcastExchange Input [2]: [customer_id#59, year_total#60] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(69) BroadcastHashJoin [codegen id : 16] +(69) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#59] Join type: Inner Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) -(70) Project [codegen id : 16] +(70) Project [codegen id : 8] Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] @@ -420,7 +424,7 @@ Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (75) +- * ColumnarToRow (74) +- CometFilter (73) @@ -443,7 +447,7 @@ Input [2]: [d_date_sk#9, d_year#10] (75) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (79) @@ -468,9 +472,9 @@ Input [2]: [d_date_sk#24, d_year#25] (79) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt index 9d3ae8fbee..ea253444c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - WholeStageCodegen (16) + WholeStageCodegen (8) Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] @@ -9,114 +9,98 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + WholeStageCodegen (2) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange #7 CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #13 + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt index a3d12a1d47..1c61a76d1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt @@ -1,133 +1,136 @@ == Physical Plan == -TakeOrderedAndProject (129) -+- * Project (128) - +- * SortMergeJoin Inner (127) - :- * Sort (71) - : +- Exchange (70) - : +- * Filter (69) - : +- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- Union (62) - : :- * Project (23) - : : +- * SortMergeJoin LeftOuter (22) - : : :- * Sort (15) - : : : +- Exchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- * Sort (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : :- * Project (42) - : : +- * SortMergeJoin LeftOuter (41) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : :- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * ColumnarToRow (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) - : : : : +- ReusedExchange (27) - : : : +- ReusedExchange (30) - : : +- * Sort (40) - : : +- Exchange (39) - : : +- * ColumnarToRow (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometScan parquet spark_catalog.default.store_returns (35) - : +- * Project (61) - : +- * SortMergeJoin LeftOuter (60) - : :- * Sort (53) - : : +- Exchange (52) - : : +- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * ColumnarToRow (45) - : : : : +- CometFilter (44) - : : : : +- CometScan parquet spark_catalog.default.web_sales (43) - : : : +- ReusedExchange (46) - : : +- ReusedExchange (49) - : +- * Sort (59) - : +- Exchange (58) - : +- * ColumnarToRow (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan parquet spark_catalog.default.web_returns (54) - +- * Sort (126) - +- Exchange (125) - +- * Filter (124) - +- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- * HashAggregate (120) - +- Exchange (119) - +- * HashAggregate (118) - +- Union (117) - :- * Project (86) - : +- * SortMergeJoin LeftOuter (85) - : :- * Sort (82) - : : +- Exchange (81) - : : +- * Project (80) - : : +- * BroadcastHashJoin Inner BuildRight (79) - : : :- * Project (77) - : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : :- * ColumnarToRow (74) - : : : : +- CometFilter (73) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) - : : : +- ReusedExchange (75) - : : +- ReusedExchange (78) - : +- * Sort (84) - : +- ReusedExchange (83) - :- * Project (101) - : +- * SortMergeJoin LeftOuter (100) - : :- * Sort (97) - : : +- Exchange (96) - : : +- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * ColumnarToRow (89) - : : : : +- CometFilter (88) - : : : : +- CometScan parquet spark_catalog.default.store_sales (87) - : : : +- ReusedExchange (90) - : : +- ReusedExchange (93) - : +- * Sort (99) - : +- ReusedExchange (98) - +- * Project (116) - +- * SortMergeJoin LeftOuter (115) - :- * Sort (112) - : +- Exchange (111) - : +- * Project (110) - : +- * BroadcastHashJoin Inner BuildRight (109) - : :- * Project (107) - : : +- * BroadcastHashJoin Inner BuildRight (106) - : : :- * ColumnarToRow (104) - : : : +- CometFilter (103) - : : : +- CometScan parquet spark_catalog.default.web_sales (102) - : : +- ReusedExchange (105) - : +- ReusedExchange (108) - +- * Sort (114) - +- ReusedExchange (113) +TakeOrderedAndProject (132) ++- * Project (131) + +- * SortMergeJoin Inner (130) + :- * Sort (72) + : +- Exchange (71) + : +- * Filter (70) + : +- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- * HashAggregate (66) + : +- Exchange (65) + : +- * HashAggregate (64) + : +- Union (63) + : :- * Project (24) + : : +- * SortMergeJoin LeftOuter (23) + : : :- * Sort (16) + : : : +- Exchange (15) + : : : +- * ColumnarToRow (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.date_dim (9) + : : +- * Sort (22) + : : +- Exchange (21) + : : +- * ColumnarToRow (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : :- * Project (43) + : : +- * SortMergeJoin LeftOuter (42) + : : :- * Sort (35) + : : : +- Exchange (34) + : : : +- * ColumnarToRow (33) + : : : +- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometFilter (26) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (25) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- * Sort (41) + : : +- Exchange (40) + : : +- * ColumnarToRow (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.store_returns (36) + : +- * Project (62) + : +- * SortMergeJoin LeftOuter (61) + : :- * Sort (54) + : : +- Exchange (53) + : : +- * ColumnarToRow (52) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometProject (48) + : : : +- CometBroadcastHashJoin (47) + : : : :- CometFilter (45) + : : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- * Sort (60) + : +- Exchange (59) + : +- * ColumnarToRow (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.web_returns (55) + +- * Sort (129) + +- Exchange (128) + +- * Filter (127) + +- * HashAggregate (126) + +- Exchange (125) + +- * HashAggregate (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- Union (120) + :- * Project (89) + : +- * SortMergeJoin LeftOuter (88) + : :- * Sort (85) + : : +- Exchange (84) + : : +- * ColumnarToRow (83) + : : +- CometProject (82) + : : +- CometBroadcastHashJoin (81) + : : :- CometProject (77) + : : : +- CometBroadcastHashJoin (76) + : : : :- CometFilter (74) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) + : : : +- ReusedExchange (75) + : : +- CometBroadcastExchange (80) + : : +- CometFilter (79) + : : +- CometScan parquet spark_catalog.default.date_dim (78) + : +- * Sort (87) + : +- ReusedExchange (86) + :- * Project (104) + : +- * SortMergeJoin LeftOuter (103) + : :- * Sort (100) + : : +- Exchange (99) + : : +- * ColumnarToRow (98) + : : +- CometProject (97) + : : +- CometBroadcastHashJoin (96) + : : :- CometProject (94) + : : : +- CometBroadcastHashJoin (93) + : : : :- CometFilter (91) + : : : : +- CometScan parquet spark_catalog.default.store_sales (90) + : : : +- ReusedExchange (92) + : : +- ReusedExchange (95) + : +- * Sort (102) + : +- ReusedExchange (101) + +- * Project (119) + +- * SortMergeJoin LeftOuter (118) + :- * Sort (115) + : +- Exchange (114) + : +- * ColumnarToRow (113) + : +- CometProject (112) + : +- CometBroadcastHashJoin (111) + : :- CometProject (109) + : : +- CometBroadcastHashJoin (108) + : : :- CometFilter (106) + : : : +- CometScan parquet spark_catalog.default.web_sales (105) + : : +- ReusedExchange (107) + : +- ReusedExchange (110) + +- * Sort (117) + +- ReusedExchange (116) (1) Scan parquet spark_catalog.default.catalog_sales @@ -142,99 +145,106 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) ColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(8) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +(8) CometProject Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 133] +(9) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +(11) CometBroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: [d_date_sk#13, d_year#14] + +(12) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Right output [2]: [d_date_sk#13, d_year#14] +Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight + +(13) CometProject Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] + +(14) ColumnarToRow [codegen id : 1] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -(14) Exchange +(15) Exchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) Sort [codegen id : 4] +(16) Sort [codegen id : 2] Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 -(16) Scan parquet spark_catalog.default.catalog_returns +(17) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(17) CometFilter +(18) CometFilter Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) -(18) CometProject +(19) CometProject Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(19) ColumnarToRow [codegen id : 5] +(20) ColumnarToRow [codegen id : 3] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(20) Exchange +(21) Exchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) Sort [codegen id : 6] +(22) Sort [codegen id : 4] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] Arguments: [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 7] +(23) SortMergeJoin [codegen id : 5] Left keys [2]: [cs_order_number#2, cs_item_sk#1] Right keys [2]: [cr_order_number#16, cr_item_sk#15] Join type: LeftOuter Join condition: None -(23) Project [codegen id : 7] +(24) Project [codegen id : 5] Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(24) Scan parquet spark_catalog.default.store_sales +(25) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] @@ -242,84 +252,82 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(25) CometFilter +(26) CometFilter Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#22) -(26) ColumnarToRow [codegen id : 10] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(27) ReusedExchange [Reuses operator id: 8] +(27) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(28) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#22] -Right keys [1]: [i_item_sk#28] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight -(29) Project [codegen id : 10] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +(29) CometProject Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(30) ReusedExchange [Reuses operator id: 133] +(30) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#33, d_year#34] -(31) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#33] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Right output [2]: [d_date_sk#33, d_year#34] +Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight -(32) Project [codegen id : 10] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +(32) CometProject Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] + +(33) ColumnarToRow [codegen id : 6] +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -(33) Exchange +(34) Exchange Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(34) Sort [codegen id : 11] +(35) Sort [codegen id : 7] Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Arguments: [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST], false, 0 -(35) Scan parquet spark_catalog.default.store_returns +(36) Scan parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(36) CometFilter +(37) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(37) CometProject +(38) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(38) ColumnarToRow [codegen id : 12] +(39) ColumnarToRow [codegen id : 8] Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(39) Exchange +(40) Exchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(40) Sort [codegen id : 13] +(41) Sort [codegen id : 9] Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST], false, 0 -(41) SortMergeJoin [codegen id : 14] +(42) SortMergeJoin [codegen id : 10] Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] Join type: LeftOuter Join condition: None -(42) Project [codegen id : 14] +(43) Project [codegen id : 10] Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(43) Scan parquet spark_catalog.default.web_sales +(44) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] @@ -327,134 +335,132 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(44) CometFilter +(45) CometFilter Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Condition : isnotnull(ws_item_sk#42) -(45) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] - -(46) ReusedExchange [Reuses operator id: 8] +(46) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(47) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#42] -Right keys [1]: [i_item_sk#48] -Join type: Inner -Join condition: None +(47) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight -(48) Project [codegen id : 17] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +(48) CometProject Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(49) ReusedExchange [Reuses operator id: 133] +(49) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#46] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(50) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight -(51) Project [codegen id : 17] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +(51) CometProject Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(52) Exchange +(52) ColumnarToRow [codegen id : 11] Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(53) Sort [codegen id : 18] +(53) Exchange +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(54) Sort [codegen id : 12] Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Arguments: [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST], false, 0 -(54) Scan parquet spark_catalog.default.web_returns +(55) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(55) CometFilter +(56) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(56) CometProject +(57) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(57) ColumnarToRow [codegen id : 19] +(58) ColumnarToRow [codegen id : 13] Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(58) Exchange +(59) Exchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(59) Sort [codegen id : 20] +(60) Sort [codegen id : 14] Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST], false, 0 -(60) SortMergeJoin [codegen id : 21] +(61) SortMergeJoin [codegen id : 15] Left keys [2]: [ws_order_number#43, ws_item_sk#42] Right keys [2]: [wr_order_number#56, wr_item_sk#55] Join type: LeftOuter Join condition: None -(61) Project [codegen id : 21] +(62) Project [codegen id : 15] Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(62) Union +(63) Union -(63) HashAggregate [codegen id : 22] +(64) HashAggregate [codegen id : 16] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(64) Exchange +(65) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(65) HashAggregate [codegen id : 23] +(66) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(66) HashAggregate [codegen id : 23] +(67) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum#62, sum#63] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -(67) Exchange +(68) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(68) HashAggregate [codegen id : 24] +(69) HashAggregate [codegen id : 18] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] -(69) Filter [codegen id : 24] +(70) Filter [codegen id : 18] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] Condition : isnotnull(sales_cnt#68) -(70) Exchange +(71) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(71) Sort [codegen id : 25] +(72) Sort [codegen id : 19] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 -(72) Scan parquet spark_catalog.default.catalog_sales +(73) Scan parquet spark_catalog.default.catalog_sales Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] @@ -462,65 +468,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(73) CometFilter +(74) CometFilter Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Condition : isnotnull(cs_item_sk#70) -(74) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] - -(75) ReusedExchange [Reuses operator id: 8] +(75) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(76) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#70] -Right keys [1]: [i_item_sk#76] -Join type: Inner -Join condition: None +(76) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Right output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70], [i_item_sk#76], Inner, BuildRight -(77) Project [codegen id : 28] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(77) CometProject Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(78) ReusedExchange [Reuses operator id: 137] +(78) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#81, d_year#82] +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 -(79) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#74] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None +(79) CometFilter +Input [2]: [d_date_sk#81, d_year#82] +Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(80) Project [codegen id : 28] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +(80) CometBroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] +Arguments: [d_date_sk#81, d_year#82] + +(81) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Right output [2]: [d_date_sk#81, d_year#82] +Arguments: [cs_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight + +(82) CometProject Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] + +(83) ColumnarToRow [codegen id : 20] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -(81) Exchange +(84) Exchange Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(82) Sort [codegen id : 29] +(85) Sort [codegen id : 21] Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 -(83) ReusedExchange [Reuses operator id: 20] +(86) ReusedExchange [Reuses operator id: 21] Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(84) Sort [codegen id : 31] +(87) Sort [codegen id : 23] Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] Arguments: [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST], false, 0 -(85) SortMergeJoin [codegen id : 32] +(88) SortMergeJoin [codegen id : 24] Left keys [2]: [cs_order_number#71, cs_item_sk#70] Right keys [2]: [cr_order_number#84, cr_item_sk#83] Join type: LeftOuter Join condition: None -(86) Project [codegen id : 32] +(89) Project [codegen id : 24] Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#87, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#88] Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(87) Scan parquet spark_catalog.default.store_sales +(90) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] Batched: true Location: InMemoryFileIndex [] @@ -528,65 +544,63 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#93), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(88) CometFilter +(91) CometFilter Input [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] Condition : isnotnull(ss_item_sk#89) -(89) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] - -(90) ReusedExchange [Reuses operator id: 8] +(92) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] -(91) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#89] -Right keys [1]: [i_item_sk#95] -Join type: Inner -Join condition: None +(93) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] +Right output [5]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] +Arguments: [ss_item_sk#89], [i_item_sk#95], Inner, BuildRight -(92) Project [codegen id : 35] -Output [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] +(94) CometProject Input [10]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] +Arguments: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99], [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] -(93) ReusedExchange [Reuses operator id: 137] +(95) ReusedExchange [Reuses operator id: 80] Output [2]: [d_date_sk#100, d_year#101] -(94) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#93] -Right keys [1]: [d_date_sk#100] -Join type: Inner -Join condition: None +(96) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] +Right output [2]: [d_date_sk#100, d_year#101] +Arguments: [ss_sold_date_sk#93], [d_date_sk#100], Inner, BuildRight -(95) Project [codegen id : 35] -Output [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] +(97) CometProject Input [11]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_date_sk#100, d_year#101] +Arguments: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101], [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] -(96) Exchange +(98) ColumnarToRow [codegen id : 25] Input [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] -Arguments: hashpartitioning(ss_ticket_number#90, ss_item_sk#89, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(97) Sort [codegen id : 36] +(99) Exchange +Input [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] +Arguments: hashpartitioning(ss_ticket_number#90, ss_item_sk#89, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(100) Sort [codegen id : 26] Input [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] Arguments: [ss_ticket_number#90 ASC NULLS FIRST, ss_item_sk#89 ASC NULLS FIRST], false, 0 -(98) ReusedExchange [Reuses operator id: 39] +(101) ReusedExchange [Reuses operator id: 40] Output [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] -(99) Sort [codegen id : 38] +(102) Sort [codegen id : 28] Input [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] Arguments: [sr_ticket_number#103 ASC NULLS FIRST, sr_item_sk#102 ASC NULLS FIRST], false, 0 -(100) SortMergeJoin [codegen id : 39] +(103) SortMergeJoin [codegen id : 29] Left keys [2]: [ss_ticket_number#90, ss_item_sk#89] Right keys [2]: [sr_ticket_number#103, sr_item_sk#102] Join type: LeftOuter Join condition: None -(101) Project [codegen id : 39] +(104) Project [codegen id : 29] Output [7]: [d_year#101, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, (ss_quantity#91 - coalesce(sr_return_quantity#104, 0)) AS sales_cnt#106, (ss_ext_sales_price#92 - coalesce(sr_return_amt#105, 0.00)) AS sales_amt#107] Input [13]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101, sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] -(102) Scan parquet spark_catalog.default.web_sales +(105) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] Batched: true Location: InMemoryFileIndex [] @@ -594,186 +608,184 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(103) CometFilter +(106) CometFilter Input [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] Condition : isnotnull(ws_item_sk#108) -(104) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] - -(105) ReusedExchange [Reuses operator id: 8] +(107) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] -(106) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#108] -Right keys [1]: [i_item_sk#114] -Join type: Inner -Join condition: None +(108) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] +Right output [5]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] +Arguments: [ws_item_sk#108], [i_item_sk#114], Inner, BuildRight -(107) Project [codegen id : 42] -Output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] +(109) CometProject Input [10]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] +Arguments: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118], [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] -(108) ReusedExchange [Reuses operator id: 137] +(110) ReusedExchange [Reuses operator id: 80] Output [2]: [d_date_sk#119, d_year#120] -(109) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#112] -Right keys [1]: [d_date_sk#119] -Join type: Inner -Join condition: None +(111) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] +Right output [2]: [d_date_sk#119, d_year#120] +Arguments: [ws_sold_date_sk#112], [d_date_sk#119], Inner, BuildRight -(110) Project [codegen id : 42] -Output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] +(112) CometProject Input [11]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_date_sk#119, d_year#120] +Arguments: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120], [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] + +(113) ColumnarToRow [codegen id : 30] +Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] -(111) Exchange +(114) Exchange Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] -Arguments: hashpartitioning(ws_order_number#109, ws_item_sk#108, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: hashpartitioning(ws_order_number#109, ws_item_sk#108, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(112) Sort [codegen id : 43] +(115) Sort [codegen id : 31] Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] Arguments: [ws_order_number#109 ASC NULLS FIRST, ws_item_sk#108 ASC NULLS FIRST], false, 0 -(113) ReusedExchange [Reuses operator id: 58] +(116) ReusedExchange [Reuses operator id: 59] Output [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] -(114) Sort [codegen id : 45] +(117) Sort [codegen id : 33] Input [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] Arguments: [wr_order_number#122 ASC NULLS FIRST, wr_item_sk#121 ASC NULLS FIRST], false, 0 -(115) SortMergeJoin [codegen id : 46] +(118) SortMergeJoin [codegen id : 34] Left keys [2]: [ws_order_number#109, ws_item_sk#108] Right keys [2]: [wr_order_number#122, wr_item_sk#121] Join type: LeftOuter Join condition: None -(116) Project [codegen id : 46] +(119) Project [codegen id : 34] Output [7]: [d_year#120, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, (ws_quantity#110 - coalesce(wr_return_quantity#123, 0)) AS sales_cnt#125, (ws_ext_sales_price#111 - coalesce(wr_return_amt#124, 0.00)) AS sales_amt#126] Input [13]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120, wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] -(117) Union +(120) Union -(118) HashAggregate [codegen id : 47] +(121) HashAggregate [codegen id : 35] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] -(119) Exchange +(122) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(120) HashAggregate [codegen id : 48] +(123) HashAggregate [codegen id : 36] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] -(121) HashAggregate [codegen id : 48] +(124) HashAggregate [codegen id : 36] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [partial_sum(sales_cnt#87), partial_sum(UnscaledValue(sales_amt#88))] Aggregate Attributes [2]: [sum#127, sum#128] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] -(122) Exchange +(125) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(123) HashAggregate [codegen id : 49] +(126) HashAggregate [codegen id : 37] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [sum(sales_cnt#87), sum(UnscaledValue(sales_amt#88))] Aggregate Attributes [2]: [sum(sales_cnt#87)#66, sum(UnscaledValue(sales_amt#88))#67] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#87)#66 AS sales_cnt#131, MakeDecimal(sum(UnscaledValue(sales_amt#88))#67,18,2) AS sales_amt#132] -(124) Filter [codegen id : 49] +(127) Filter [codegen id : 37] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] Condition : isnotnull(sales_cnt#131) -(125) Exchange +(128) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] -Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(126) Sort [codegen id : 50] +(129) Sort [codegen id : 38] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 -(127) SortMergeJoin [codegen id : 51] +(130) SortMergeJoin [codegen id : 39] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Join type: Inner Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#131 as decimal(17,2))) < 0.90000000000000000000) -(128) Project [codegen id : 51] +(131) Project [codegen id : 39] Output [10]: [d_year#82 AS prev_year#133, d_year#14 AS year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#131 AS prev_yr_cnt#135, sales_cnt#68 AS curr_yr_cnt#136, (sales_cnt#68 - sales_cnt#131) AS sales_cnt_diff#137, (sales_amt#69 - sales_amt#132) AS sales_amt_diff#138] Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] -(129) TakeOrderedAndProject +(132) TakeOrderedAndProject Input [10]: [prev_year#133, year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#135, curr_yr_cnt#136, sales_cnt_diff#137, sales_amt_diff#138] Arguments: 100, [sales_cnt_diff#137 ASC NULLS FIRST], [prev_year#133, year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#135, curr_yr_cnt#136, sales_cnt_diff#137, sales_amt_diff#138] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (133) -+- * ColumnarToRow (132) - +- CometFilter (131) - +- CometScan parquet spark_catalog.default.date_dim (130) +BroadcastExchange (136) ++- * ColumnarToRow (135) + +- CometFilter (134) + +- CometScan parquet spark_catalog.default.date_dim (133) -(130) Scan parquet spark_catalog.default.date_dim +(133) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(131) CometFilter +(134) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(132) ColumnarToRow [codegen id : 1] +(135) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(133) BroadcastExchange +(136) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 -BroadcastExchange (137) -+- * ColumnarToRow (136) - +- CometFilter (135) - +- CometScan parquet spark_catalog.default.date_dim (134) +Subquery:4 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (140) ++- * ColumnarToRow (139) + +- CometFilter (138) + +- CometScan parquet spark_catalog.default.date_dim (137) -(134) Scan parquet spark_catalog.default.date_dim +(137) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#81, d_year#82] 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 -(135) CometFilter +(138) CometFilter Input [2]: [d_date_sk#81, d_year#82] Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(136) ColumnarToRow [codegen id : 1] +(139) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] -(137) BroadcastExchange +(140) BroadcastExchange Input [2]: [d_date_sk#81, d_year#82] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#75 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt index 44bcabcdb1..34866bc37e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt @@ -1,41 +1,41 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_amt_diff] - WholeStageCodegen (51) + WholeStageCodegen (39) Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter - WholeStageCodegen (25) + WholeStageCodegen (19) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (24) + WholeStageCodegen (18) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - WholeStageCodegen (23) + WholeStageCodegen (17) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - WholeStageCodegen (22) + WholeStageCodegen (16) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (7) + WholeStageCodegen (5) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (2) Sort [cs_order_number,cs_item_sk] InputAdapter Exchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -45,196 +45,185 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometBroadcastExchange #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometBroadcastExchange #7 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (4) Sort [cr_order_number,cr_item_sk] InputAdapter - Exchange [cr_order_number,cr_item_sk] #7 - WholeStageCodegen (5) + Exchange [cr_order_number,cr_item_sk] #8 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - WholeStageCodegen (14) + WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (7) Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #9 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (9) Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #9 - WholeStageCodegen (12) + Exchange [sr_ticket_number,sr_item_sk] #10 + WholeStageCodegen (8) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - WholeStageCodegen (21) + WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (18) + WholeStageCodegen (12) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (17) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ws_order_number,ws_item_sk] #11 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter - WholeStageCodegen (20) + WholeStageCodegen (14) Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_order_number,wr_item_sk] #11 - WholeStageCodegen (19) + Exchange [wr_order_number,wr_item_sk] #12 + WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter - WholeStageCodegen (50) + WholeStageCodegen (38) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - WholeStageCodegen (49) + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + WholeStageCodegen (37) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (48) + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + WholeStageCodegen (36) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - WholeStageCodegen (47) + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + WholeStageCodegen (35) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (32) + WholeStageCodegen (24) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (29) + WholeStageCodegen (21) Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (28) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [cs_order_number,cs_item_sk] #16 + WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 + BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange #18 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (31) + WholeStageCodegen (23) Sort [cr_order_number,cr_item_sk] InputAdapter - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - WholeStageCodegen (39) + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + WholeStageCodegen (29) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (36) + WholeStageCodegen (26) Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (35) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #19 + WholeStageCodegen (25) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter - WholeStageCodegen (38) + WholeStageCodegen (28) Sort [sr_ticket_number,sr_item_sk] InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - WholeStageCodegen (46) + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + WholeStageCodegen (34) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (43) + WholeStageCodegen (31) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (42) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ws_order_number,ws_item_sk] #20 + WholeStageCodegen (30) + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter - WholeStageCodegen (45) + WholeStageCodegen (33) Sort [wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt index f3b80d8bdf..088855406e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt @@ -1,42 +1,38 @@ == Physical Plan == -TakeOrderedAndProject (38) -+- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- Union (34) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (13) - : +- * ColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.date_dim (10) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.web_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * ColumnarToRow (27) - : : +- CometFilter (26) - : : +- CometScan parquet spark_catalog.default.catalog_sales (25) - : +- ReusedExchange (28) - +- ReusedExchange (31) +TakeOrderedAndProject (34) ++- * HashAggregate (33) + +- Exchange (32) + +- * HashAggregate (31) + +- * ColumnarToRow (30) + +- CometUnion (29) + :- CometProject (12) + : +- CometBroadcastHashJoin (11) + : :- CometProject (7) + : : +- CometBroadcastHashJoin (6) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.item (3) + : +- CometBroadcastExchange (10) + : +- CometFilter (9) + : +- CometScan parquet spark_catalog.default.date_dim (8) + :- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.web_sales (13) + : : +- ReusedExchange (15) + : +- ReusedExchange (18) + +- CometProject (28) + +- CometBroadcastHashJoin (27) + :- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.catalog_sales (21) + : +- ReusedExchange (23) + +- ReusedExchange (26) (1) Scan parquet spark_catalog.default.store_sales @@ -51,66 +47,55 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [i_item_sk#5, i_category#6] Condition : isnotnull(i_item_sk#5) -(6) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_category#6] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [2]: [i_item_sk#5, i_category#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#5, i_category#6] -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Right output [2]: [i_item_sk#5, i_category#6] +Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) Project [codegen id : 3] -Output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6] +(7) CometProject Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#6] +Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6], [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6] -(10) Scan parquet spark_catalog.default.date_dim +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Condition : isnotnull(d_date_sk#7) -(12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [d_date_sk#7, d_year#8, d_qoy#9] -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6] +Right output [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Arguments: [ss_sold_date_sk#4], [d_date_sk#7], Inner, BuildRight -(15) Project [codegen id : 3] -Output [6]: [store AS channel#10, ss_store_sk#2 AS col_name#11, d_year#8, d_qoy#9, i_category#6, ss_ext_sales_price#3 AS ext_sales_price#12] +(12) CometProject Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6, d_date_sk#7, d_year#8, d_qoy#9] +Arguments: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12], [store AS channel#10, ss_store_sk#2 AS col_name#11, d_year#8, d_qoy#9, i_category#6, ss_ext_sales_price#3 AS ext_sales_price#12] -(16) Scan parquet spark_catalog.default.web_sales +(13) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] @@ -118,40 +103,35 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#16)] PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] Condition : (isnull(ws_ship_customer_sk#14) AND isnotnull(ws_item_sk#13)) -(18) ColumnarToRow [codegen id : 6] -Input [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] - -(19) ReusedExchange [Reuses operator id: 7] +(15) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#17, i_category#18] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#13] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] +Right output [2]: [i_item_sk#17, i_category#18] +Arguments: [ws_item_sk#13], [i_item_sk#17], Inner, BuildRight -(21) Project [codegen id : 6] -Output [4]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18] +(17) CometProject Input [6]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_item_sk#17, i_category#18] +Arguments: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18], [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18] -(22) ReusedExchange [Reuses operator id: 13] +(18) ReusedExchange [Reuses operator id: 10] Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [4]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18] +Right output [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Arguments: [ws_sold_date_sk#16], [d_date_sk#19], Inner, BuildRight -(24) Project [codegen id : 6] -Output [6]: [web AS channel#22, ws_ship_customer_sk#14 AS col_name#23, d_year#20, d_qoy#21, i_category#18, ws_ext_sales_price#15 AS ext_sales_price#24] +(20) CometProject Input [7]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18, d_date_sk#19, d_year#20, d_qoy#21] +Arguments: [channel#22, col_name#23, d_year#20, d_qoy#21, i_category#18, ext_sales_price#24], [web AS channel#22, ws_ship_customer_sk#14 AS col_name#23, d_year#20, d_qoy#21, i_category#18, ws_ext_sales_price#15 AS ext_sales_price#24] -(25) Scan parquet spark_catalog.default.catalog_sales +(21) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] @@ -159,60 +139,61 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#28)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(26) CometFilter +(22) CometFilter Input [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] Condition : (isnull(cs_ship_addr_sk#25) AND isnotnull(cs_item_sk#26)) -(27) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] - -(28) ReusedExchange [Reuses operator id: 7] +(23) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#29, i_category#30] -(29) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#26] -Right keys [1]: [i_item_sk#29] -Join type: Inner -Join condition: None +(24) CometBroadcastHashJoin +Left output [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] +Right output [2]: [i_item_sk#29, i_category#30] +Arguments: [cs_item_sk#26], [i_item_sk#29], Inner, BuildRight -(30) Project [codegen id : 9] -Output [4]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30] +(25) CometProject Input [6]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28, i_item_sk#29, i_category#30] +Arguments: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30], [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30] -(31) ReusedExchange [Reuses operator id: 13] +(26) ReusedExchange [Reuses operator id: 10] Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#28] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [4]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30] +Right output [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Arguments: [cs_sold_date_sk#28], [d_date_sk#31], Inner, BuildRight -(33) Project [codegen id : 9] -Output [6]: [catalog AS channel#34, cs_ship_addr_sk#25 AS col_name#35, d_year#32, d_qoy#33, i_category#30, cs_ext_sales_price#27 AS ext_sales_price#36] +(28) CometProject Input [7]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30, d_date_sk#31, d_year#32, d_qoy#33] +Arguments: [channel#34, col_name#35, d_year#32, d_qoy#33, i_category#30, ext_sales_price#36], [catalog AS channel#34, cs_ship_addr_sk#25 AS col_name#35, d_year#32, d_qoy#33, i_category#30, cs_ext_sales_price#27 AS ext_sales_price#36] + +(29) CometUnion +Child 0 Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12] +Child 1 Input [6]: [channel#22, col_name#23, d_year#20, d_qoy#21, i_category#18, ext_sales_price#24] +Child 2 Input [6]: [channel#34, col_name#35, d_year#32, d_qoy#33, i_category#30, ext_sales_price#36] -(34) Union +(30) ColumnarToRow [codegen id : 1] +Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12] -(35) HashAggregate [codegen id : 10] +(31) HashAggregate [codegen id : 1] Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12] Keys [5]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#12))] Aggregate Attributes [2]: [count#37, sum#38] Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] -(36) Exchange +(32) Exchange Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] -Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(37) HashAggregate [codegen id : 11] +(33) HashAggregate [codegen id : 2] Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] Keys [5]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] Aggregate Attributes [2]: [count(1)#41, sum(UnscaledValue(ext_sales_price#12))#42] Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count(1)#41 AS sales_cnt#43, MakeDecimal(sum(UnscaledValue(ext_sales_price#12))#42,17,2) AS sales_amt#44] -(38) TakeOrderedAndProject +(34) TakeOrderedAndProject Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] Arguments: 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt index 73e6b09afe..6f8c28004e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt @@ -1,58 +1,38 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - WholeStageCodegen (11) + WholeStageCodegen (2) HashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] [count(1),sum(UnscaledValue(ext_sales_price)),sales_cnt,sales_amt,count,sum] InputAdapter Exchange [channel,col_name,d_year,d_qoy,i_category] #1 - WholeStageCodegen (10) + WholeStageCodegen (1) HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] - InputAdapter - Union - WholeStageCodegen (3) - Project [ss_store_sk,d_year,d_qoy,i_category,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - WholeStageCodegen (6) - Project [ws_ship_customer_sk,d_year,d_qoy,i_category,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_ship_customer_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] - InputAdapter - ReusedExchange [i_item_sk,i_category] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - WholeStageCodegen (9) - Project [cs_ship_addr_sk,d_year,d_qoy,i_category,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_ship_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - ReusedExchange [i_item_sk,i_category] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 + ColumnarToRow + InputAdapter + CometUnion + CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometBroadcastExchange #3 + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_ship_customer_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_ship_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt index bbfa6a4c4a..c177fb453d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt @@ -1,89 +1,90 @@ == Physical Plan == -TakeOrderedAndProject (85) -+- * HashAggregate (84) - +- Exchange (83) - +- * HashAggregate (82) - +- * Expand (81) - +- Union (80) - :- * Project (30) - : +- * BroadcastHashJoin LeftOuter BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.store_returns (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - :- * Project (49) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (48) - : :- BroadcastExchange (39) - : : +- * HashAggregate (38) - : : +- Exchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * ColumnarToRow (32) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : +- ReusedExchange (33) - : +- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * ColumnarToRow (41) - : : +- CometScan parquet spark_catalog.default.catalog_returns (40) - : +- ReusedExchange (42) - +- * Project (79) - +- * BroadcastHashJoin LeftOuter BuildRight (78) - :- * HashAggregate (64) - : +- Exchange (63) - : +- * HashAggregate (62) - : +- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Project (55) - : : +- * BroadcastHashJoin Inner BuildRight (54) - : : :- * ColumnarToRow (52) - : : : +- CometFilter (51) - : : : +- CometScan parquet spark_catalog.default.web_sales (50) - : : +- ReusedExchange (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_page (56) - +- BroadcastExchange (77) - +- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * ColumnarToRow (67) - : : +- CometFilter (66) - : : +- CometScan parquet spark_catalog.default.web_returns (65) - : +- ReusedExchange (68) - +- ReusedExchange (71) +TakeOrderedAndProject (86) ++- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Expand (82) + +- Union (81) + :- * Project (32) + : +- * BroadcastHashJoin LeftOuter BuildRight (31) + : :- * HashAggregate (17) + : : +- Exchange (16) + : : +- * HashAggregate (15) + : : +- * ColumnarToRow (14) + : : +- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- BroadcastExchange (30) + : +- * HashAggregate (29) + : +- Exchange (28) + : +- * HashAggregate (27) + : +- * ColumnarToRow (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.store_returns (18) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + :- * Project (51) + : +- * BroadcastNestedLoopJoin Inner BuildLeft (50) + : :- BroadcastExchange (41) + : : +- * HashAggregate (40) + : : +- Exchange (39) + : : +- * HashAggregate (38) + : : +- * ColumnarToRow (37) + : : +- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometScan parquet spark_catalog.default.catalog_sales (33) + : : +- ReusedExchange (34) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * HashAggregate (47) + : +- * ColumnarToRow (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometScan parquet spark_catalog.default.catalog_returns (42) + : +- ReusedExchange (43) + +- * Project (80) + +- * BroadcastHashJoin LeftOuter BuildRight (79) + :- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * ColumnarToRow (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (56) + : : +- CometBroadcastHashJoin (55) + : : :- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.web_sales (52) + : : +- ReusedExchange (54) + : +- CometBroadcastExchange (59) + : +- CometFilter (58) + : +- CometScan parquet spark_catalog.default.web_page (57) + +- BroadcastExchange (78) + +- * HashAggregate (77) + +- Exchange (76) + +- * HashAggregate (75) + +- * ColumnarToRow (74) + +- CometProject (73) + +- CometBroadcastHashJoin (72) + :- CometProject (70) + : +- CometBroadcastHashJoin (69) + : :- CometFilter (67) + : : +- CometScan parquet spark_catalog.default.web_returns (66) + : +- ReusedExchange (68) + +- ReusedExchange (71) (1) Scan parquet spark_catalog.default.store_sales @@ -98,450 +99,450 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_date#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +(7) CometBroadcastHashJoin +Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -(7) Scan parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] +(9) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) +(10) CometFilter +Input [1]: [s_store_sk#8] +Condition : isnotnull(s_store_sk#8) -(9) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] +(11) CometBroadcastExchange +Input [1]: [s_store_sk#8] +Arguments: [s_store_sk#8] -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +Right output [1]: [s_store_sk#8] +Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None +(13) CometProject +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +(14) ColumnarToRow [codegen id : 1] +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] +(15) HashAggregate [codegen id : 1] +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Keys [1]: [s_store_sk#8] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] +Aggregate Attributes [2]: [sum#9, sum#10] +Results [3]: [s_store_sk#8, sum#11, sum#12] -(14) Exchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) Exchange +Input [3]: [s_store_sk#8, sum#11, sum#12] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] +(17) HashAggregate [codegen id : 4] +Input [3]: [s_store_sk#8, sum#11, sum#12] +Keys [1]: [s_store_sk#8] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#13, sum(UnscaledValue(ss_net_profit#3))#14] +Results [3]: [s_store_sk#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#13,17,2) AS sales#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#14,17,2) AS profit#16] -(16) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +(18) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, sr_returned_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(sr_returned_date_sk#20), dynamicpruningexpression(sr_returned_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(17) CometFilter -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) - -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] - -(19) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#21] - -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] - -(22) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#22] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] - -(25) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Keys [1]: [s_store_sk#22] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#23, sum#24] -Results [3]: [s_store_sk#22, sum#25, sum#26] - -(26) Exchange -Input [3]: [s_store_sk#22, sum#25, sum#26] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#22, sum#25, sum#26] -Keys [1]: [s_store_sk#22] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] -Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] - -(28) BroadcastExchange -Input [3]: [s_store_sk#22, returns#29, profit_loss#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#22] +(19) CometFilter +Input [4]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, sr_returned_date_sk#20] +Condition : isnotnull(sr_store_sk#17) + +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#22] + +(21) CometBroadcastHashJoin +Left output [4]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, sr_returned_date_sk#20] +Right output [1]: [d_date_sk#22] +Arguments: [sr_returned_date_sk#20], [d_date_sk#22], Inner, BuildRight + +(22) CometProject +Input [5]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, sr_returned_date_sk#20, d_date_sk#22] +Arguments: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19], [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19] + +(23) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#23] + +(24) CometBroadcastHashJoin +Left output [3]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19] +Right output [1]: [s_store_sk#23] +Arguments: [sr_store_sk#17], [s_store_sk#23], Inner, BuildRight + +(25) CometProject +Input [4]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, s_store_sk#23] +Arguments: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23], [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] + +(26) ColumnarToRow [codegen id : 2] +Input [3]: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] + +(27) HashAggregate [codegen id : 2] +Input [3]: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] +Keys [1]: [s_store_sk#23] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#18)), partial_sum(UnscaledValue(sr_net_loss#19))] +Aggregate Attributes [2]: [sum#24, sum#25] +Results [3]: [s_store_sk#23, sum#26, sum#27] + +(28) Exchange +Input [3]: [s_store_sk#23, sum#26, sum#27] +Arguments: hashpartitioning(s_store_sk#23, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(29) HashAggregate [codegen id : 3] +Input [3]: [s_store_sk#23, sum#26, sum#27] +Keys [1]: [s_store_sk#23] +Functions [2]: [sum(UnscaledValue(sr_return_amt#18)), sum(UnscaledValue(sr_net_loss#19))] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#18))#28, sum(UnscaledValue(sr_net_loss#19))#29] +Results [3]: [s_store_sk#23, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#28,17,2) AS returns#30, MakeDecimal(sum(UnscaledValue(sr_net_loss#19))#29,17,2) AS profit_loss#31] + +(30) BroadcastExchange +Input [3]: [s_store_sk#23, returns#30, profit_loss#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(31) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [s_store_sk#8] +Right keys [1]: [s_store_sk#23] Join type: LeftOuter Join condition: None -(30) Project [codegen id : 8] -Output [5]: [sales#14, coalesce(returns#29, 0.00) AS returns#31, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#32, store channel AS channel#33, s_store_sk#7 AS id#34] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] +(32) Project [codegen id : 4] +Output [5]: [sales#15, coalesce(returns#30, 0.00) AS returns#32, (profit#16 - coalesce(profit_loss#31, 0.00)) AS profit#33, store channel AS channel#34, s_store_sk#8 AS id#35] +Input [6]: [s_store_sk#8, sales#15, profit#16, s_store_sk#23, returns#30, profit_loss#31] -(31) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(33) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38, cs_sold_date_sk#39] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(cs_sold_date_sk#39), dynamicpruningexpression(cs_sold_date_sk#39 IN dynamicpruning#40)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] - -(33) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#40] - -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] - -(36) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum#41, sum#42] -Results [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(37) Exchange -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(38) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] -Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] - -(39) BroadcastExchange -Input [3]: [cs_call_center_sk#35, sales#47, profit#48] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(40) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(34) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#41] + +(35) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38, cs_sold_date_sk#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#39], [d_date_sk#41], Inner, BuildRight + +(36) CometProject +Input [5]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38, cs_sold_date_sk#39, d_date_sk#41] +Arguments: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38], [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] + +(37) ColumnarToRow [codegen id : 5] +Input [3]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] + +(38) HashAggregate [codegen id : 5] +Input [3]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] +Keys [1]: [cs_call_center_sk#36] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#37)), partial_sum(UnscaledValue(cs_net_profit#38))] +Aggregate Attributes [2]: [sum#42, sum#43] +Results [3]: [cs_call_center_sk#36, sum#44, sum#45] + +(39) Exchange +Input [3]: [cs_call_center_sk#36, sum#44, sum#45] +Arguments: hashpartitioning(cs_call_center_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(40) HashAggregate [codegen id : 6] +Input [3]: [cs_call_center_sk#36, sum#44, sum#45] +Keys [1]: [cs_call_center_sk#36] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#37)), sum(UnscaledValue(cs_net_profit#38))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#37))#46, sum(UnscaledValue(cs_net_profit#38))#47] +Results [3]: [cs_call_center_sk#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#37))#46,17,2) AS sales#48, MakeDecimal(sum(UnscaledValue(cs_net_profit#38))#47,17,2) AS profit#49] + +(41) BroadcastExchange +Input [3]: [cs_call_center_sk#36, sales#48, profit#49] +Arguments: IdentityBroadcastMode, [plan_id=5] + +(42) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(cr_returned_date_sk#52), dynamicpruningexpression(cr_returned_date_sk#52 IN dynamicpruning#53)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(43) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#54] -(42) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#53] +(44) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +Right output [1]: [d_date_sk#54] +Arguments: [cr_returned_date_sk#52], [d_date_sk#54], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#51] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(45) CometProject +Input [4]: [cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52, d_date_sk#54] +Arguments: [cr_return_amount#50, cr_net_loss#51], [cr_return_amount#50, cr_net_loss#51] -(44) Project [codegen id : 13] -Output [2]: [cr_return_amount#49, cr_net_loss#50] -Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] +(46) ColumnarToRow [codegen id : 7] +Input [2]: [cr_return_amount#50, cr_net_loss#51] -(45) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#49, cr_net_loss#50] +(47) HashAggregate [codegen id : 7] +Input [2]: [cr_return_amount#50, cr_net_loss#51] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum#54, sum#55] -Results [2]: [sum#56, sum#57] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#50)), partial_sum(UnscaledValue(cr_net_loss#51))] +Aggregate Attributes [2]: [sum#55, sum#56] +Results [2]: [sum#57, sum#58] -(46) Exchange -Input [2]: [sum#56, sum#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(48) Exchange +Input [2]: [sum#57, sum#58] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(47) HashAggregate -Input [2]: [sum#56, sum#57] +(49) HashAggregate +Input [2]: [sum#57, sum#58] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] +Functions [2]: [sum(UnscaledValue(cr_return_amount#50)), sum(UnscaledValue(cr_net_loss#51))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#50))#59, sum(UnscaledValue(cr_net_loss#51))#60] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#50))#59,17,2) AS returns#61, MakeDecimal(sum(UnscaledValue(cr_net_loss#51))#60,17,2) AS profit_loss#62] -(48) BroadcastNestedLoopJoin [codegen id : 14] +(50) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(49) Project [codegen id : 14] -Output [5]: [sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#62, catalog channel AS channel#63, cs_call_center_sk#35 AS id#64] -Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] +(51) Project [codegen id : 8] +Output [5]: [sales#48, returns#61, (profit#49 - profit_loss#62) AS profit#63, catalog channel AS channel#64, cs_call_center_sk#36 AS id#65] +Input [5]: [cs_call_center_sk#36, sales#48, profit#49, returns#61, profit_loss#62] -(50) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(52) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, ws_sold_date_sk#69] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(51) CometFilter -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_web_page_sk#65) +(53) CometFilter +Input [4]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, ws_sold_date_sk#69] +Condition : isnotnull(ws_web_page_sk#66) -(52) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(54) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#71] -(53) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#70] +(55) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, ws_sold_date_sk#69] +Right output [1]: [d_date_sk#71] +Arguments: [ws_sold_date_sk#69], [d_date_sk#71], Inner, BuildRight -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#68] -Right keys [1]: [d_date_sk#70] -Join type: Inner -Join condition: None - -(55) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] -Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] +(56) CometProject +Input [5]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, ws_sold_date_sk#69, d_date_sk#71] +Arguments: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68], [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68] -(56) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#71] +(57) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(57) CometFilter -Input [1]: [wp_web_page_sk#71] -Condition : isnotnull(wp_web_page_sk#71) - -(58) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#71] - -(59) BroadcastExchange -Input [1]: [wp_web_page_sk#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(60) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#65] -Right keys [1]: [wp_web_page_sk#71] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] - -(62) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum#72, sum#73] -Results [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(63) Exchange -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(64) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] -Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] - -(65) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(58) CometFilter +Input [1]: [wp_web_page_sk#72] +Condition : isnotnull(wp_web_page_sk#72) + +(59) CometBroadcastExchange +Input [1]: [wp_web_page_sk#72] +Arguments: [wp_web_page_sk#72] + +(60) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68] +Right output [1]: [wp_web_page_sk#72] +Arguments: [ws_web_page_sk#66], [wp_web_page_sk#72], Inner, BuildRight + +(61) CometProject +Input [4]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] +Arguments: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72], [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] + +(62) ColumnarToRow [codegen id : 9] +Input [3]: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] + +(63) HashAggregate [codegen id : 9] +Input [3]: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] +Keys [1]: [wp_web_page_sk#72] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#67)), partial_sum(UnscaledValue(ws_net_profit#68))] +Aggregate Attributes [2]: [sum#73, sum#74] +Results [3]: [wp_web_page_sk#72, sum#75, sum#76] + +(64) Exchange +Input [3]: [wp_web_page_sk#72, sum#75, sum#76] +Arguments: hashpartitioning(wp_web_page_sk#72, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(65) HashAggregate [codegen id : 12] +Input [3]: [wp_web_page_sk#72, sum#75, sum#76] +Keys [1]: [wp_web_page_sk#72] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#67)), sum(UnscaledValue(ws_net_profit#68))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#67))#77, sum(UnscaledValue(ws_net_profit#68))#78] +Results [3]: [wp_web_page_sk#72, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#67))#77,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(ws_net_profit#68))#78,17,2) AS profit#80] + +(66) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wr_returned_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(wr_returned_date_sk#84), dynamicpruningexpression(wr_returned_date_sk#84 IN dynamicpruning#85)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(66) CometFilter -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] -Condition : isnotnull(wr_web_page_sk#80) - -(67) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(67) CometFilter +Input [4]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wr_returned_date_sk#84] +Condition : isnotnull(wr_web_page_sk#81) -(68) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#85] +(68) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#86] -(69) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#83] -Right keys [1]: [d_date_sk#85] -Join type: Inner -Join condition: None +(69) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wr_returned_date_sk#84] +Right output [1]: [d_date_sk#86] +Arguments: [wr_returned_date_sk#84], [d_date_sk#86], Inner, BuildRight -(70) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] -Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] +(70) CometProject +Input [5]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wr_returned_date_sk#84, d_date_sk#86] +Arguments: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83], [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83] (71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#86] - -(72) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#80] -Right keys [1]: [wp_web_page_sk#86] -Join type: Inner -Join condition: None - -(73) Project [codegen id : 20] -Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] - -(74) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum#87, sum#88] -Results [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(75) Exchange -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(76) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] -Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] - -(77) BroadcastExchange -Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(78) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#71] -Right keys [1]: [wp_web_page_sk#86] +Output [1]: [wp_web_page_sk#87] + +(72) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83] +Right output [1]: [wp_web_page_sk#87] +Arguments: [wr_web_page_sk#81], [wp_web_page_sk#87], Inner, BuildRight + +(73) CometProject +Input [4]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] +Arguments: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87], [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] + +(74) ColumnarToRow [codegen id : 10] +Input [3]: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] + +(75) HashAggregate [codegen id : 10] +Input [3]: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] +Keys [1]: [wp_web_page_sk#87] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#82)), partial_sum(UnscaledValue(wr_net_loss#83))] +Aggregate Attributes [2]: [sum#88, sum#89] +Results [3]: [wp_web_page_sk#87, sum#90, sum#91] + +(76) Exchange +Input [3]: [wp_web_page_sk#87, sum#90, sum#91] +Arguments: hashpartitioning(wp_web_page_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(77) HashAggregate [codegen id : 11] +Input [3]: [wp_web_page_sk#87, sum#90, sum#91] +Keys [1]: [wp_web_page_sk#87] +Functions [2]: [sum(UnscaledValue(wr_return_amt#82)), sum(UnscaledValue(wr_net_loss#83))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#82))#92, sum(UnscaledValue(wr_net_loss#83))#93] +Results [3]: [wp_web_page_sk#87, MakeDecimal(sum(UnscaledValue(wr_return_amt#82))#92,17,2) AS returns#94, MakeDecimal(sum(UnscaledValue(wr_net_loss#83))#93,17,2) AS profit_loss#95] + +(78) BroadcastExchange +Input [3]: [wp_web_page_sk#87, returns#94, profit_loss#95] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +(79) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [wp_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#87] Join type: LeftOuter Join condition: None -(79) Project [codegen id : 22] -Output [5]: [sales#78, coalesce(returns#93, 0.00) AS returns#95, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#96, web channel AS channel#97, wp_web_page_sk#71 AS id#98] -Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] +(80) Project [codegen id : 12] +Output [5]: [sales#79, coalesce(returns#94, 0.00) AS returns#96, (profit#80 - coalesce(profit_loss#95, 0.00)) AS profit#97, web channel AS channel#98, wp_web_page_sk#72 AS id#99] +Input [6]: [wp_web_page_sk#72, sales#79, profit#80, wp_web_page_sk#87, returns#94, profit_loss#95] -(80) Union +(81) Union -(81) Expand [codegen id : 23] -Input [5]: [sales#14, returns#31, profit#32, channel#33, id#34] -Arguments: [[sales#14, returns#31, profit#32, channel#33, id#34, 0], [sales#14, returns#31, profit#32, channel#33, null, 1], [sales#14, returns#31, profit#32, null, null, 3]], [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] +(82) Expand [codegen id : 13] +Input [5]: [sales#15, returns#32, profit#33, channel#34, id#35] +Arguments: [[sales#15, returns#32, profit#33, channel#34, id#35, 0], [sales#15, returns#32, profit#33, channel#34, null, 1], [sales#15, returns#32, profit#33, null, null, 3]], [sales#15, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] -(82) HashAggregate [codegen id : 23] -Input [6]: [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] -Keys [3]: [channel#99, id#100, spark_grouping_id#101] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#31), partial_sum(profit#32)] -Aggregate Attributes [6]: [sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] -Results [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +(83) HashAggregate [codegen id : 13] +Input [6]: [sales#15, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] +Keys [3]: [channel#100, id#101, spark_grouping_id#102] +Functions [3]: [partial_sum(sales#15), partial_sum(returns#32), partial_sum(profit#33)] +Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Results [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -(83) Exchange -Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Arguments: hashpartitioning(channel#99, id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(84) Exchange +Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: hashpartitioning(channel#100, id#101, spark_grouping_id#102, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(84) HashAggregate [codegen id : 24] -Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Keys [3]: [channel#99, id#100, spark_grouping_id#101] -Functions [3]: [sum(sales#14), sum(returns#31), sum(profit#32)] -Aggregate Attributes [3]: [sum(sales#14)#114, sum(returns#31)#115, sum(profit#32)#116] -Results [5]: [channel#99, id#100, sum(sales#14)#114 AS sales#117, sum(returns#31)#115 AS returns#118, sum(profit#32)#116 AS profit#119] +(85) HashAggregate [codegen id : 14] +Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [3]: [channel#100, id#101, spark_grouping_id#102] +Functions [3]: [sum(sales#15), sum(returns#32), sum(profit#33)] +Aggregate Attributes [3]: [sum(sales#15)#115, sum(returns#32)#116, sum(profit#33)#117] +Results [5]: [channel#100, id#101, sum(sales#15)#115 AS sales#118, sum(returns#32)#116 AS returns#119, sum(profit#33)#117 AS profit#120] -(85) TakeOrderedAndProject -Input [5]: [channel#99, id#100, sales#117, returns#118, profit#119] -Arguments: 100, [channel#99 ASC NULLS FIRST, id#100 ASC NULLS FIRST], [channel#99, id#100, sales#117, returns#118, profit#119] +(86) TakeOrderedAndProject +Input [5]: [channel#100, id#101, sales#118, returns#119, profit#120] +Arguments: 100, [channel#100 ASC NULLS FIRST, id#101 ASC NULLS FIRST], [channel#100, id#101, sales#118, returns#119, profit#120] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (90) -+- * ColumnarToRow (89) - +- CometProject (88) - +- CometFilter (87) - +- CometScan parquet spark_catalog.default.date_dim (86) +BroadcastExchange (91) ++- * ColumnarToRow (90) + +- CometProject (89) + +- CometFilter (88) + +- CometScan parquet spark_catalog.default.date_dim (87) -(86) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#120] +(87) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_date#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter -Input [2]: [d_date_sk#6, d_date#120] -Condition : (((isnotnull(d_date#120) AND (d_date#120 >= 2000-08-03)) AND (d_date#120 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) +(88) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) -(88) CometProject -Input [2]: [d_date_sk#6, d_date#120] +(89) CometProject +Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(89) ColumnarToRow [codegen id : 1] +(90) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(90) BroadcastExchange +(91) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#20 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#39 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 40 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#52 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 65 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#84 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt index d6693067f0..2989455d98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt @@ -1,27 +1,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (24) + WholeStageCodegen (14) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (23) + WholeStageCodegen (13) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (8) + WholeStageCodegen (4) Project [sales,returns,profit,profit_loss,s_store_sk] BroadcastHashJoin [s_store_sk,s_store_sk] HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [s_store_sk] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -32,112 +32,99 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #5 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (3) HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [s_store_sk] #6 - WholeStageCodegen (6) + Exchange [s_store_sk] #7 + WholeStageCodegen (2) HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometProject [sr_store_sk,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [s_store_sk] #4 - WholeStageCodegen (14) + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (8) Project [sales,returns,profit,profit_loss,cs_call_center_sk] BroadcastNestedLoopJoin InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) + BroadcastExchange #8 + WholeStageCodegen (6) HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [cs_call_center_sk] #8 - WholeStageCodegen (10) + Exchange [cs_call_center_sk] #9 + WholeStageCodegen (5) HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange #9 - WholeStageCodegen (13) + Exchange #10 + WholeStageCodegen (7) HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (12) Project [sales,returns,profit,profit_loss,wp_web_page_sk] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #10 - WholeStageCodegen (17) + Exchange [wp_web_page_sk] #11 + WholeStageCodegen (9) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #12 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (21) + BroadcastExchange #13 + WholeStageCodegen (11) HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #13 - WholeStageCodegen (20) + Exchange [wp_web_page_sk] #14 + WholeStageCodegen (10) HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] CometFilter [wr_web_page_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [wp_web_page_sk] #11 + ReusedExchange [d_date_sk] #4 + ReusedExchange [wp_web_page_sk] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt index c89bad2201..810eaa6e2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt @@ -1,34 +1,35 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * BroadcastHashJoin Inner BuildRight (28) - :- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (27) - +- * ColumnarToRow (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.customer (24) +TakeOrderedAndProject (31) ++- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * HashAggregate (24) + : +- Exchange (23) + : +- * HashAggregate (22) + : +- * ColumnarToRow (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.household_demographics (15) + +- BroadcastExchange (28) + +- * ColumnarToRow (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (1) Scan parquet spark_catalog.default.store_sales @@ -43,166 +44,173 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#10] +(4) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Arguments: [d_date_sk#10], [d_date_sk#10] -(6) Project [codegen id : 4] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] + +(7) CometBroadcastHashJoin +Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight + +(8) CometProject Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -(7) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#11, s_number_employees#12, s_city#13] +(9) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#13, s_number_employees#14, s_city#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Condition : (((isnotnull(s_number_employees#12) AND (s_number_employees#12 >= 200)) AND (s_number_employees#12 <= 295)) AND isnotnull(s_store_sk#11)) - -(9) CometProject -Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Arguments: [s_store_sk#11, s_city#13], [s_store_sk#11, s_city#13] +(10) CometFilter +Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] +Condition : (((isnotnull(s_number_employees#14) AND (s_number_employees#14 >= 200)) AND (s_number_employees#14 <= 295)) AND isnotnull(s_store_sk#13)) -(10) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_city#13] +(11) CometProject +Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] +Arguments: [s_store_sk#13, s_city#15], [s_store_sk#13, s_city#15] -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_city#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [2]: [s_store_sk#13, s_city#15] +Arguments: [s_store_sk#13, s_city#15] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +Right output [2]: [s_store_sk#13, s_city#15] +Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight -(13) Project [codegen id : 4] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11, s_city#13] +(14) CometProject +Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13, s_city#15] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -(14) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] +(15) Scan parquet spark_catalog.default.household_demographics +Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : (((hd_dep_count#15 = 6) OR (hd_vehicle_count#16 > 2)) AND isnotnull(hd_demo_sk#14)) +(16) CometFilter +Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] +Condition : (((hd_dep_count#17 = 6) OR (hd_vehicle_count#18 > 2)) AND isnotnull(hd_demo_sk#16)) -(16) CometProject -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Arguments: [hd_demo_sk#14], [hd_demo_sk#14] +(17) CometProject +Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] +Arguments: [hd_demo_sk#16], [hd_demo_sk#16] -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#14] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#16] +Arguments: [hd_demo_sk#16] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(19) CometBroadcastHashJoin +Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] +Right output [1]: [hd_demo_sk#16] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] -Join type: Inner -Join condition: None +(20) CometProject +Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15, hd_demo_sk#16] +Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -(20) Project [codegen id : 4] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13, hd_demo_sk#14] +(21) ColumnarToRow [codegen id : 1] +Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -(21) HashAggregate [codegen id : 4] -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] +(22) HashAggregate [codegen id : 1] +Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum#17, sum#18] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] +Aggregate Attributes [2]: [sum#19, sum#20] +Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#21, sum#22] -(22) Exchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(23) Exchange +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#21, sum#22] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(23) HashAggregate [codegen id : 6] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] +(24) HashAggregate [codegen id : 3] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#21, sum#22] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#21, sum(UnscaledValue(ss_net_profit#7))#22] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#21,17,2) AS amt#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#22,17,2) AS profit#24] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#23, sum(UnscaledValue(ss_net_profit#7))#24] +Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#23,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#24,17,2) AS profit#26] -(24) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +(25) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#27, c_first_name#28, c_last_name#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(25) CometFilter -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Condition : isnotnull(c_customer_sk#25) +(26) CometFilter +Input [3]: [c_customer_sk#27, c_first_name#28, c_last_name#29] +Condition : isnotnull(c_customer_sk#27) -(26) ColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +(27) ColumnarToRow [codegen id : 2] +Input [3]: [c_customer_sk#27, c_first_name#28, c_last_name#29] -(27) BroadcastExchange -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(28) BroadcastExchange +Input [3]: [c_customer_sk#27, c_first_name#28, c_last_name#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(28) BroadcastHashJoin [codegen id : 6] +(29) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#25] +Right keys [1]: [c_customer_sk#27] Join type: Inner Join condition: None -(29) Project [codegen id : 6] -Output [7]: [c_last_name#27, c_first_name#26, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#26, c_last_name#27] +(30) Project [codegen id : 3] +Output [7]: [c_last_name#29, c_first_name#28, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#25, profit#26, s_city#15] +Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#25, profit#26, c_customer_sk#27, c_first_name#28, c_last_name#29] -(30) TakeOrderedAndProject -Input [7]: [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Arguments: 100, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24] +(31) TakeOrderedAndProject +Input [7]: [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#25, profit#26, s_city#15] +Arguments: 100, [c_last_name#29 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#26 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#25, profit#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (36) ++- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(31) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#29, d_dow#30] +(32) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_dow#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [d_date_sk#10, d_year#29, d_dow#30] -Condition : (((isnotnull(d_dow#30) AND (d_dow#30 = 1)) AND d_year#29 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) +(33) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(33) CometProject -Input [3]: [d_date_sk#10, d_year#29, d_dow#30] +(34) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(34) ColumnarToRow [codegen id : 1] +(35) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(35) BroadcastExchange +(36) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt index 4c05c449c5..5fcbcfed39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] InputAdapter Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -25,27 +25,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, CometProject [d_date_sk] CometFilter [d_dow,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_dow,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange #4 + CometProject [s_store_sk,s_city] + CometFilter [s_number_employees,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] + CometBroadcastExchange #5 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) + BroadcastExchange #6 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt index 7d6d717c1b..07bfd877c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt @@ -1,47 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (37) - +- * HashAggregate (36) - +- Exchange (35) - +- * HashAggregate (34) - +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * ColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.customer_address (13) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * ColumnarToRow (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.customer_address (17) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan parquet spark_catalog.default.customer (20) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * ColumnarToRow (14) + : +- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.store (9) + +- BroadcastExchange (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * BroadcastHashJoin LeftSemi BuildRight (34) + :- * ColumnarToRow (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.customer_address (15) + +- BroadcastExchange (33) + +- * Project (32) + +- * Filter (31) + +- * HashAggregate (30) + +- Exchange (29) + +- * ColumnarToRow (28) + +- CometHashAggregate (27) + +- CometProject (26) + +- CometBroadcastHashJoin (25) + :- CometFilter (20) + : +- CometScan parquet spark_catalog.default.customer_address (19) + +- CometBroadcastExchange (24) + +- CometProject (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.customer (21) (1) Scan parquet spark_catalog.default.store_sales @@ -56,233 +57,238 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(7) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_store_name#7, s_zip#8] +(9) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#8, s_store_name#9, s_zip#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(8) CometFilter -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Condition : (isnotnull(s_store_sk#6) AND isnotnull(s_zip#8)) +(10) CometFilter +Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] +Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_zip#10)) -(9) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] +(11) CometBroadcastExchange +Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] +Arguments: [s_store_sk#8, s_store_name#9, s_zip#10] -(10) BroadcastExchange -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [2]: [ss_store_sk#1, ss_net_profit#2] +Right output [3]: [s_store_sk#8, s_store_name#9, s_zip#10] +Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None +(13) CometProject +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_store_name#9, s_zip#10] +Arguments: [ss_net_profit#2, s_store_name#9, s_zip#10], [ss_net_profit#2, s_store_name#9, s_zip#10] -(12) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#8] +(14) ColumnarToRow [codegen id : 5] +Input [3]: [ss_net_profit#2, s_store_name#9, s_zip#10] -(13) Scan parquet spark_catalog.default.customer_address -Output [1]: [ca_zip#9] +(15) Scan parquet spark_catalog.default.customer_address +Output [1]: [ca_zip#11] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] ReadSchema: struct -(14) CometFilter -Input [1]: [ca_zip#9] -Condition : (substr(ca_zip#9, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#9, 1, 5))) +(16) CometFilter +Input [1]: [ca_zip#11] +Condition : (substr(ca_zip#11, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#11, 1, 5))) -(15) CometProject -Input [1]: [ca_zip#9] -Arguments: [ca_zip#10], [substr(ca_zip#9, 1, 5) AS ca_zip#10] +(17) CometProject +Input [1]: [ca_zip#11] +Arguments: [ca_zip#12], [substr(ca_zip#11, 1, 5) AS ca_zip#12] -(16) ColumnarToRow [codegen id : 6] -Input [1]: [ca_zip#10] +(18) ColumnarToRow [codegen id : 3] +Input [1]: [ca_zip#12] -(17) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#11, ca_zip#12] +(19) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#13, ca_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(18) CometFilter -Input [2]: [ca_address_sk#11, ca_zip#12] -Condition : isnotnull(ca_address_sk#11) - -(19) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#11, ca_zip#12] +(20) CometFilter +Input [2]: [ca_address_sk#13, ca_zip#14] +Condition : isnotnull(ca_address_sk#13) -(20) Scan parquet spark_catalog.default.customer -Output [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] +(21) Scan parquet spark_catalog.default.customer +Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(21) CometFilter -Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] -Condition : ((isnotnull(c_preferred_cust_flag#14) AND (c_preferred_cust_flag#14 = Y)) AND isnotnull(c_current_addr_sk#13)) +(22) CometFilter +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) -(22) CometProject -Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] -Arguments: [c_current_addr_sk#13], [c_current_addr_sk#13] +(23) CometProject +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Arguments: [c_current_addr_sk#15], [c_current_addr_sk#15] -(23) ColumnarToRow [codegen id : 3] -Input [1]: [c_current_addr_sk#13] +(24) CometBroadcastExchange +Input [1]: [c_current_addr_sk#15] +Arguments: [c_current_addr_sk#15] -(24) BroadcastExchange -Input [1]: [c_current_addr_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(25) CometBroadcastHashJoin +Left output [2]: [ca_address_sk#13, ca_zip#14] +Right output [1]: [c_current_addr_sk#15] +Arguments: [ca_address_sk#13], [c_current_addr_sk#15], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ca_address_sk#11] -Right keys [1]: [c_current_addr_sk#13] -Join type: Inner -Join condition: None +(26) CometProject +Input [3]: [ca_address_sk#13, ca_zip#14, c_current_addr_sk#15] +Arguments: [ca_zip#14], [ca_zip#14] -(26) Project [codegen id : 4] -Output [1]: [ca_zip#12] -Input [3]: [ca_address_sk#11, ca_zip#12, c_current_addr_sk#13] - -(27) HashAggregate [codegen id : 4] -Input [1]: [ca_zip#12] -Keys [1]: [ca_zip#12] +(27) CometHashAggregate +Input [1]: [ca_zip#14] +Keys [1]: [ca_zip#14] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [2]: [ca_zip#12, count#16] -(28) Exchange -Input [2]: [ca_zip#12, count#16] -Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(28) ColumnarToRow [codegen id : 1] +Input [2]: [ca_zip#14, count#17] -(29) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#12, count#16] -Keys [1]: [ca_zip#12] +(29) Exchange +Input [2]: [ca_zip#14, count#17] +Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(30) HashAggregate [codegen id : 2] +Input [2]: [ca_zip#14, count#17] +Keys [1]: [ca_zip#14] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [2]: [substr(ca_zip#12, 1, 5) AS ca_zip#18, count(1)#17 AS cnt#19] +Aggregate Attributes [1]: [count(1)#18] +Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#19, count(1)#18 AS cnt#20] -(30) Filter [codegen id : 5] -Input [2]: [ca_zip#18, cnt#19] -Condition : (cnt#19 > 10) +(31) Filter [codegen id : 2] +Input [2]: [ca_zip#19, cnt#20] +Condition : (cnt#20 > 10) -(31) Project [codegen id : 5] -Output [1]: [ca_zip#18] -Input [2]: [ca_zip#18, cnt#19] +(32) Project [codegen id : 2] +Output [1]: [ca_zip#19] +Input [2]: [ca_zip#19, cnt#20] -(32) BroadcastExchange -Input [1]: [ca_zip#18] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [plan_id=4] +(33) BroadcastExchange +Input [1]: [ca_zip#19] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [plan_id=2] -(33) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [coalesce(ca_zip#10, ), isnull(ca_zip#10)] -Right keys [2]: [coalesce(ca_zip#18, ), isnull(ca_zip#18)] +(34) BroadcastHashJoin [codegen id : 3] +Left keys [2]: [coalesce(ca_zip#12, ), isnull(ca_zip#12)] +Right keys [2]: [coalesce(ca_zip#19, ), isnull(ca_zip#19)] Join type: LeftSemi Join condition: None -(34) HashAggregate [codegen id : 6] -Input [1]: [ca_zip#10] -Keys [1]: [ca_zip#10] +(35) HashAggregate [codegen id : 3] +Input [1]: [ca_zip#12] +Keys [1]: [ca_zip#12] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#10] +Results [1]: [ca_zip#12] -(35) Exchange -Input [1]: [ca_zip#10] -Arguments: hashpartitioning(ca_zip#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(36) Exchange +Input [1]: [ca_zip#12] +Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(36) HashAggregate [codegen id : 7] -Input [1]: [ca_zip#10] -Keys [1]: [ca_zip#10] +(37) HashAggregate [codegen id : 4] +Input [1]: [ca_zip#12] +Keys [1]: [ca_zip#12] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#10] +Results [1]: [ca_zip#12] -(37) BroadcastExchange -Input [1]: [ca_zip#10] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=6] +(38) BroadcastExchange +Input [1]: [ca_zip#12] +Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=4] -(38) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [substr(s_zip#8, 1, 2)] -Right keys [1]: [substr(ca_zip#10, 1, 2)] +(39) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [substr(s_zip#10, 1, 2)] +Right keys [1]: [substr(ca_zip#12, 1, 2)] Join type: Inner Join condition: None -(39) Project [codegen id : 8] -Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#10] +(40) Project [codegen id : 5] +Output [2]: [ss_net_profit#2, s_store_name#9] +Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#10, ca_zip#12] -(40) HashAggregate [codegen id : 8] -Input [2]: [ss_net_profit#2, s_store_name#7] -Keys [1]: [s_store_name#7] +(41) HashAggregate [codegen id : 5] +Input [2]: [ss_net_profit#2, s_store_name#9] +Keys [1]: [s_store_name#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#20] -Results [2]: [s_store_name#7, sum#21] +Aggregate Attributes [1]: [sum#21] +Results [2]: [s_store_name#9, sum#22] -(41) Exchange -Input [2]: [s_store_name#7, sum#21] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(42) Exchange +Input [2]: [s_store_name#9, sum#22] +Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(42) HashAggregate [codegen id : 9] -Input [2]: [s_store_name#7, sum#21] -Keys [1]: [s_store_name#7] +(43) HashAggregate [codegen id : 6] +Input [2]: [s_store_name#9, sum#22] +Keys [1]: [s_store_name#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS sum(ss_net_profit)#23] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] +Results [2]: [s_store_name#9, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS sum(ss_net_profit)#24] -(43) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#23] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#23] +(44) TakeOrderedAndProject +Input [2]: [s_store_name#9, sum(ss_net_profit)#24] +Arguments: 100, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#24] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(44) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#24, d_qoy#25] +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2)) AND (d_year#24 = 1998)) AND isnotnull(d_date_sk#5)) +(46) CometFilter +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) -(46) CometProject -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] +(47) CometProject +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(47) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(48) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt index 76fa276931..adf5554172 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (9) + WholeStageCodegen (6) HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] InputAdapter Exchange [s_store_name] #1 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [s_store_name,ss_net_profit] [sum,sum] Project [ss_net_profit,s_store_name] BroadcastHashJoin [s_zip,ca_zip] - Project [ss_net_profit,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_net_profit,s_store_name,s_zip] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -23,22 +23,20 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] CometProject [d_date_sk] CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #4 + CometFilter [s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) + BroadcastExchange #5 + WholeStageCodegen (4) HashAggregate [ca_zip] InputAdapter - Exchange [ca_zip] #5 - WholeStageCodegen (6) + Exchange [ca_zip] #6 + WholeStageCodegen (3) HashAggregate [ca_zip] BroadcastHashJoin [ca_zip,ca_zip] ColumnarToRow @@ -47,26 +45,22 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] CometFilter [ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_zip] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (2) Project [ca_zip] Filter [cnt] HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] InputAdapter - Exchange [ca_zip] #7 - WholeStageCodegen (4) - HashAggregate [ca_zip] [count,count] - Project [ca_zip] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter + Exchange [ca_zip] #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_zip] + CometProject [ca_zip] + CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] + CometBroadcastExchange #9 + CometProject [c_current_addr_sk] + CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt index 431d1453dd..b266f4a241 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt @@ -1,52 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (33) - : : +- * Filter (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * HashAggregate (28) - : : +- Exchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : +- BroadcastExchange (39) - : +- * ColumnarToRow (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.customer (36) - +- BroadcastExchange (45) - +- * ColumnarToRow (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.customer_address (42) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (43) + : +- * BroadcastHashJoin Inner BuildRight (42) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- Exchange (16) + : : : +- * HashAggregate (15) + : : : +- * ColumnarToRow (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.customer_address (9) + : : +- BroadcastExchange (35) + : : +- * Filter (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * HashAggregate (31) + : : +- * HashAggregate (30) + : : +- Exchange (29) + : : +- * HashAggregate (28) + : : +- * ColumnarToRow (27) + : : +- CometProject (26) + : : +- CometBroadcastHashJoin (25) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (19) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : +- BroadcastExchange (41) + : +- * ColumnarToRow (40) + : +- CometFilter (39) + : +- CometScan parquet spark_catalog.default.customer (38) + +- BroadcastExchange (47) + +- * ColumnarToRow (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.customer_address (44) (1) Scan parquet spark_catalog.default.catalog_returns @@ -61,259 +63,268 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_year#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(6) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] +(7) CometBroadcastHashJoin +Left output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [cr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] +Arguments: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3], [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -(7) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_state#8] +(9) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#8, ca_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] ReadSchema: struct -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_state#8] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(ca_state#8)) +(10) CometFilter +Input [2]: [ca_address_sk#8, ca_state#9] +Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_state#9)) -(9) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#7, ca_state#8] +(11) CometBroadcastExchange +Input [2]: [ca_address_sk#8, ca_state#9] +Arguments: [ca_address_sk#8, ca_state#9] -(10) BroadcastExchange -Input [2]: [ca_address_sk#7, ca_state#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] +Right output [2]: [ca_address_sk#8, ca_state#9] +Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(13) CometProject +Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#9] +Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -(12) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#8] +(14) ColumnarToRow [codegen id : 1] +Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -(13) HashAggregate [codegen id : 3] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] -Keys [2]: [cr_returning_customer_sk#1, ca_state#8] +(15) HashAggregate [codegen id : 1] +Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] +Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#9] -Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] +Aggregate Attributes [1]: [sum#10] +Results [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] -(14) Exchange -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) Exchange +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 11] -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#8] +(17) HashAggregate [codegen id : 7] +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] +Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#11] -Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#12] +Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#12,17,2) AS ctr_total_return#15] -(16) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) +(18) Filter [codegen id : 7] +Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] +Condition : isnotnull(ctr_total_return#15) -(17) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] +(19) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#18), dynamicpruningexpression(cr_returned_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(cr_returned_date_sk#19), dynamicpruningexpression(cr_returned_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(18) CometFilter -Input [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Condition : isnotnull(cr_returning_addr_sk#16) - -(19) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] - -(20) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#20] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#18] -Right keys [1]: [d_date_sk#20] +(20) CometFilter +Input [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] +Condition : isnotnull(cr_returning_addr_sk#17) + +(21) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#21] + +(22) CometBroadcastHashJoin +Left output [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] +Right output [1]: [d_date_sk#21] +Arguments: [cr_returned_date_sk#19], [d_date_sk#21], Inner, BuildRight + +(23) CometProject +Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19, d_date_sk#21] +Arguments: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18], [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18] + +(24) ReusedExchange [Reuses operator id: 11] +Output [2]: [ca_address_sk#22, ca_state#23] + +(25) CometBroadcastHashJoin +Left output [3]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18] +Right output [2]: [ca_address_sk#22, ca_state#23] +Arguments: [cr_returning_addr_sk#17], [ca_address_sk#22], Inner, BuildRight + +(26) CometProject +Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, ca_address_sk#22, ca_state#23] +Arguments: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23], [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23] + +(27) ColumnarToRow [codegen id : 2] +Input [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23] + +(28) HashAggregate [codegen id : 2] +Input [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23] +Keys [2]: [cr_returning_customer_sk#16, ca_state#23] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#18))] +Aggregate Attributes [1]: [sum#24] +Results [3]: [cr_returning_customer_sk#16, ca_state#23, sum#25] + +(29) Exchange +Input [3]: [cr_returning_customer_sk#16, ca_state#23, sum#25] +Arguments: hashpartitioning(cr_returning_customer_sk#16, ca_state#23, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(30) HashAggregate [codegen id : 3] +Input [3]: [cr_returning_customer_sk#16, ca_state#23, sum#25] +Keys [2]: [cr_returning_customer_sk#16, ca_state#23] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))#12] +Results [2]: [ca_state#23 AS ctr_state#26, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#18))#12,17,2) AS ctr_total_return#27] + +(31) HashAggregate [codegen id : 3] +Input [2]: [ctr_state#26, ctr_total_return#27] +Keys [1]: [ctr_state#26] +Functions [1]: [partial_avg(ctr_total_return#27)] +Aggregate Attributes [2]: [sum#28, count#29] +Results [3]: [ctr_state#26, sum#30, count#31] + +(32) Exchange +Input [3]: [ctr_state#26, sum#30, count#31] +Arguments: hashpartitioning(ctr_state#26, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(33) HashAggregate [codegen id : 4] +Input [3]: [ctr_state#26, sum#30, count#31] +Keys [1]: [ctr_state#26] +Functions [1]: [avg(ctr_total_return#27)] +Aggregate Attributes [1]: [avg(ctr_total_return#27)#32] +Results [2]: [(avg(ctr_total_return#27)#32 * 1.2) AS (avg(ctr_total_return) * 1.2)#33, ctr_state#26] + +(34) Filter [codegen id : 4] +Input [2]: [(avg(ctr_total_return) * 1.2)#33, ctr_state#26] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#33) + +(35) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#33, ctr_state#26] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] + +(36) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ctr_state#14] +Right keys [1]: [ctr_state#26] Join type: Inner -Join condition: None +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#33) -(22) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] -Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18, d_date_sk#20] +(37) Project [codegen id : 7] +Output [2]: [ctr_customer_sk#13, ctr_total_return#15] +Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#33, ctr_state#26] -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#21, ca_state#22] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#16] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] -Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, ca_address_sk#21, ca_state#22] - -(26) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] -Keys [2]: [cr_returning_customer_sk#15, ca_state#22] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#17))] -Aggregate Attributes [1]: [sum#23] -Results [3]: [cr_returning_customer_sk#15, ca_state#22, sum#24] - -(27) Exchange -Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#24] -Arguments: hashpartitioning(cr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(28) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#24] -Keys [2]: [cr_returning_customer_sk#15, ca_state#22] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#17))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#17))#11] -Results [2]: [ca_state#22 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#17))#11,17,2) AS ctr_total_return#26] - -(29) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#25, ctr_total_return#26] -Keys [1]: [ctr_state#25] -Functions [1]: [partial_avg(ctr_total_return#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [3]: [ctr_state#25, sum#29, count#30] - -(30) Exchange -Input [3]: [ctr_state#25, sum#29, count#30] -Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#25, sum#29, count#30] -Keys [1]: [ctr_state#25] -Functions [1]: [avg(ctr_total_return#26)] -Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] -Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(32) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) - -(33) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#13] -Right keys [1]: [ctr_state#25] -Join type: Inner -Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) - -(35) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(36) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] +(38) Scan parquet spark_catalog.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 -(37) CometFilter -Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) +(39) CometFilter +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)) -(38) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] +(40) ColumnarToRow [codegen id : 5] +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] -(39) BroadcastExchange -Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(41) BroadcastExchange +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] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#12] -Right keys [1]: [c_customer_sk#33] +(42) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ctr_customer_sk#13] +Right keys [1]: [c_customer_sk#34] Join type: Inner Join condition: None -(41) Project [codegen id : 11] -Output [6]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] -Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] +(43) Project [codegen id : 7] +Output [6]: [ctr_total_return#15, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] +Input [8]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] -(42) Scan parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#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#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +(44) Scan parquet spark_catalog.default.customer_address +Output [12]: [ca_address_sk#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#47, ca_zip#48, ca_country#49, ca_gmt_offset#50, ca_location_type#51] 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 -(43) CometFilter -Input [12]: [ca_address_sk#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#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Condition : ((isnotnull(ca_state#46) AND (ca_state#46 = GA)) AND isnotnull(ca_address_sk#39)) +(45) CometFilter +Input [12]: [ca_address_sk#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#47, ca_zip#48, ca_country#49, ca_gmt_offset#50, ca_location_type#51] +Condition : ((isnotnull(ca_state#47) AND (ca_state#47 = GA)) AND isnotnull(ca_address_sk#40)) -(44) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#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#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +(46) ColumnarToRow [codegen id : 6] +Input [12]: [ca_address_sk#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#47, ca_zip#48, ca_country#49, ca_gmt_offset#50, ca_location_type#51] -(45) BroadcastExchange -Input [12]: [ca_address_sk#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#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(47) BroadcastExchange +Input [12]: [ca_address_sk#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#47, ca_zip#48, ca_country#49, ca_gmt_offset#50, ca_location_type#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#39] +(48) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#36] +Right keys [1]: [ca_address_sk#40] Join type: Inner Join condition: None -(47) Project [codegen id : 11] -Output [16]: [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#14] -Input [18]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_address_sk#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#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +(49) Project [codegen id : 7] +Output [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#47, ca_zip#48, ca_country#49, ca_gmt_offset#50, ca_location_type#51, ctr_total_return#15] +Input [18]: [ctr_total_return#15, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, ca_address_sk#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#47, ca_zip#48, ca_country#49, ca_gmt_offset#50, ca_location_type#51] -(48) TakeOrderedAndProject -Input [16]: [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#14] -Arguments: 100, [c_customer_id#34 ASC NULLS FIRST, c_salutation#36 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, c_last_name#38 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#46 ASC NULLS FIRST, ca_zip#47 ASC NULLS FIRST, ca_country#48 ASC NULLS FIRST, ca_gmt_offset#49 ASC NULLS FIRST, ca_location_type#50 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#14] +(50) TakeOrderedAndProject +Input [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#47, ca_zip#48, ca_country#49, ca_gmt_offset#50, ca_location_type#51, ctr_total_return#15] +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#41 ASC NULLS FIRST, ca_street_name#42 ASC NULLS FIRST, ca_street_type#43 ASC NULLS FIRST, ca_suite_number#44 ASC NULLS FIRST, ca_city#45 ASC NULLS FIRST, ca_county#46 ASC NULLS FIRST, ca_state#47 ASC NULLS FIRST, ca_zip#48 ASC NULLS FIRST, ca_country#49 ASC NULLS FIRST, ca_gmt_offset#50 ASC NULLS FIRST, ca_location_type#51 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#47, ca_zip#48, ca_country#49, ca_gmt_offset#50, ca_location_type#51, ctr_total_return#15] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (55) ++- * ColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) -(49) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#51] +(51) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_year#7] 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 -(50) CometFilter -Input [2]: [d_date_sk#6, d_year#51] -Condition : ((isnotnull(d_year#51) AND (d_year#51 = 2000)) AND isnotnull(d_date_sk#6)) +(52) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(51) CometProject -Input [2]: [d_date_sk#6, d_year#51] +(53) CometProject +Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(52) ColumnarToRow [codegen id : 1] +(54) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(53) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 17 Hosting Expression = cr_returned_date_sk#18 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#19 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt index 6f042847ca..3daee59f85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt @@ -1,5 +1,5 @@ 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 (11) + WholeStageCodegen (7) 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 [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] @@ -10,14 +10,14 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st 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] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) 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] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -28,52 +28,48 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) + BroadcastExchange #5 + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] InputAdapter - Exchange [ctr_state] #5 - WholeStageCodegen (7) + Exchange [ctr_state] #6 + WholeStageCodegen (3) 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] #6 - WholeStageCodegen (6) + Exchange [cr_returning_customer_sk,ca_state] #7 + WholeStageCodegen (2) 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] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometFilter [cr_returning_addr_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) + BroadcastExchange #8 + WholeStageCodegen (5) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) + BroadcastExchange #9 + WholeStageCodegen (6) ColumnarToRow InputAdapter CometFilter [ca_state,ca_address_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt index 7609fa5208..00e38f6948 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt @@ -1,29 +1,30 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * ColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan parquet spark_catalog.default.store_sales (16) +TakeOrderedAndProject (26) ++- * HashAggregate (25) + +- Exchange (24) + +- * ColumnarToRow (23) + +- CometHashAggregate (22) + +- CometProject (21) + +- CometBroadcastHashJoin (20) + :- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- CometBroadcastExchange (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.inventory (4) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.date_dim (10) + +- CometProject (19) + +- CometFilter (18) + +- CometScan parquet spark_catalog.default.store_sales (17) (1) Scan parquet spark_catalog.default.item @@ -41,10 +42,7 @@ Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) A Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(4) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] - -(5) Scan parquet spark_catalog.default.inventory +(4) Scan parquet spark_catalog.default.inventory Output [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -52,128 +50,136 @@ PartitionFilters: [isnotnull(inv_date_sk#8), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Condition : (((isnotnull(inv_quantity_on_hand#7) AND (inv_quantity_on_hand#7 >= 100)) AND (inv_quantity_on_hand#7 <= 500)) AND isnotnull(inv_item_sk#6)) -(7) CometProject +(6) CometProject Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Arguments: [inv_item_sk#6, inv_date_sk#8], [inv_item_sk#6, inv_date_sk#8] -(8) ColumnarToRow [codegen id : 1] -Input [2]: [inv_item_sk#6, inv_date_sk#8] - -(9) BroadcastExchange +(7) CometBroadcastExchange Input [2]: [inv_item_sk#6, inv_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [inv_item_sk#6, inv_date_sk#8] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#6] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Right output [2]: [inv_item_sk#6, inv_date_sk#8] +Arguments: [i_item_sk#1], [inv_item_sk#6], Inner, BuildRight -(11) Project [codegen id : 3] -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] +(9) CometProject Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_item_sk#6, inv_date_sk#8] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] -(12) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#10] +(10) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_date#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [inv_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(11) CometFilter +Input [2]: [d_date_sk#10, d_date#11] +Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-05-25)) AND (d_date#11 <= 2000-07-24)) AND isnotnull(d_date_sk#10)) -(14) Project [codegen id : 3] -Output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +(12) CometProject +Input [2]: [d_date_sk#10, d_date#11] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] + +(14) CometBroadcastHashJoin +Left output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [inv_date_sk#8], [d_date_sk#10], Inner, BuildRight + +(15) CometProject Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8, d_date_sk#10] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(15) BroadcastExchange +(16) CometBroadcastExchange Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(16) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] +(17) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_item_sk#12, ss_sold_date_sk#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(17) CometFilter -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_item_sk#11) - -(18) CometProject -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#11], [ss_item_sk#11] +(18) CometFilter +Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_item_sk#12) -(19) ColumnarToRow -Input [1]: [ss_item_sk#11] +(19) CometProject +Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] +Arguments: [ss_item_sk#12], [ss_item_sk#12] -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#11] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Right output [1]: [ss_item_sk#12] +Arguments: [i_item_sk#1], [ss_item_sk#12], Inner, BuildLeft -(21) Project [codegen id : 4] -Output [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, ss_item_sk#11] +(21) CometProject +Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, ss_item_sk#12] +Arguments: [i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_id#2, i_item_desc#3, i_current_price#4] -(22) HashAggregate [codegen id : 4] +(22) CometHashAggregate Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(23) Exchange +(23) ColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) HashAggregate [codegen id : 5] +(24) Exchange +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(25) HashAggregate [codegen id : 2] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] Aggregate Attributes: [] Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(25) TakeOrderedAndProject +(26) TakeOrderedAndProject Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (30) -+- * ColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.date_dim (26) +Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(26) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_date#13] +(27) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_date#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter -Input [2]: [d_date_sk#10, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-05-25)) AND (d_date#13 <= 2000-07-24)) AND isnotnull(d_date_sk#10)) +(28) CometFilter +Input [2]: [d_date_sk#10, d_date#11] +Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-05-25)) AND (d_date#11 <= 2000-07-24)) AND isnotnull(d_date_sk#10)) -(28) CometProject -Input [2]: [d_date_sk#10, d_date#13] +(29) CometProject +Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(29) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt index 0252eb575f..a4d96e6f14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt @@ -1,44 +1,38 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_current_price] InputAdapter Exchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometProject [i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastExchange #2 + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometBroadcastExchange #3 + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt index 5183e02755..d9d3198136 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt @@ -1,50 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (46) -+- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_returns (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (10) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- BroadcastExchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometFilter (32) - : : +- CometScan parquet spark_catalog.default.web_returns (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * HashAggregate (19) + : : +- Exchange (18) + : : +- * HashAggregate (17) + : : +- * ColumnarToRow (16) + : : +- CometProject (15) + : : +- CometBroadcastHashJoin (14) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_returns (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.item (3) + : : +- CometBroadcastExchange (13) + : : +- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- ReusedExchange (10) + : +- BroadcastExchange (32) + : +- * HashAggregate (31) + : +- Exchange (30) + : +- * HashAggregate (29) + : +- * ColumnarToRow (28) + : +- CometProject (27) + : +- CometBroadcastHashJoin (26) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (20) + : : +- ReusedExchange (22) + : +- ReusedExchange (25) + +- BroadcastExchange (47) + +- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * ColumnarToRow (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.web_returns (35) + : +- ReusedExchange (37) + +- ReusedExchange (40) (1) Scan parquet spark_catalog.default.store_returns @@ -59,314 +63,319 @@ ReadSchema: struct Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) -(3) ColumnarToRow [codegen id : 5] -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#5, i_item_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [i_item_sk#5, i_item_id#6] Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [2]: [i_item_sk#5, i_item_id#6] +Arguments: [i_item_sk#5, i_item_id#6] -(7) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] +Right output [2]: [i_item_sk#5, i_item_id#6] +Arguments: [sr_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 5] -Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] +(7) CometProject Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#6] +Arguments: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6], [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] -(10) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#7] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_date#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) -(12) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#6] +(10) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date#9] + +(11) CometBroadcastHashJoin +Left output [2]: [d_date_sk#7, d_date#8] +Right output [1]: [d_date#9] +Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight + +(12) CometProject +Input [2]: [d_date_sk#7, d_date#8] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(14) CometBroadcastHashJoin +Left output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] +Right output [1]: [d_date_sk#7] +Arguments: [sr_returned_date_sk#3], [d_date_sk#7], Inner, BuildRight + +(15) CometProject Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6, d_date_sk#7] +Arguments: [sr_return_quantity#2, i_item_id#6], [sr_return_quantity#2, i_item_id#6] -(13) HashAggregate [codegen id : 5] +(16) ColumnarToRow [codegen id : 1] +Input [2]: [sr_return_quantity#2, i_item_id#6] + +(17) HashAggregate [codegen id : 1] Input [2]: [sr_return_quantity#2, i_item_id#6] Keys [1]: [i_item_id#6] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#8] -Results [2]: [i_item_id#6, sum#9] +Aggregate Attributes [1]: [sum#10] +Results [2]: [i_item_id#6, sum#11] -(14) Exchange -Input [2]: [i_item_id#6, sum#9] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(18) Exchange +Input [2]: [i_item_id#6, sum#11] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#6, sum#9] +(19) HashAggregate [codegen id : 6] +Input [2]: [i_item_id#6, sum#11] Keys [1]: [i_item_id#6] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#10] -Results [2]: [i_item_id#6 AS item_id#11, sum(sr_return_quantity#2)#10 AS sr_item_qty#12] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#12] +Results [2]: [i_item_id#6 AS item_id#13, sum(sr_return_quantity#2)#12 AS sr_item_qty#14] -(16) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] +(20) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#15), dynamicpruningexpression(cr_returned_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cr_returned_date_sk#17), dynamicpruningexpression(cr_returned_date_sk#17 IN dynamicpruning#18)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(17) CometFilter -Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] -Condition : isnotnull(cr_item_sk#13) - -(18) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] - -(19) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#17, i_item_id#18] - -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#13] -Right keys [1]: [i_item_sk#17] +(21) CometFilter +Input [3]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17] +Condition : isnotnull(cr_item_sk#15) + +(22) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#19, i_item_id#20] + +(23) CometBroadcastHashJoin +Left output [3]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17] +Right output [2]: [i_item_sk#19, i_item_id#20] +Arguments: [cr_item_sk#15], [i_item_sk#19], Inner, BuildRight + +(24) CometProject +Input [5]: [cr_item_sk#15, cr_return_quantity#16, cr_returned_date_sk#17, i_item_sk#19, i_item_id#20] +Arguments: [cr_return_quantity#16, cr_returned_date_sk#17, i_item_id#20], [cr_return_quantity#16, cr_returned_date_sk#17, i_item_id#20] + +(25) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#21] + +(26) CometBroadcastHashJoin +Left output [3]: [cr_return_quantity#16, cr_returned_date_sk#17, i_item_id#20] +Right output [1]: [d_date_sk#21] +Arguments: [cr_returned_date_sk#17], [d_date_sk#21], Inner, BuildRight + +(27) CometProject +Input [4]: [cr_return_quantity#16, cr_returned_date_sk#17, i_item_id#20, d_date_sk#21] +Arguments: [cr_return_quantity#16, i_item_id#20], [cr_return_quantity#16, i_item_id#20] + +(28) ColumnarToRow [codegen id : 2] +Input [2]: [cr_return_quantity#16, i_item_id#20] + +(29) HashAggregate [codegen id : 2] +Input [2]: [cr_return_quantity#16, i_item_id#20] +Keys [1]: [i_item_id#20] +Functions [1]: [partial_sum(cr_return_quantity#16)] +Aggregate Attributes [1]: [sum#22] +Results [2]: [i_item_id#20, sum#23] + +(30) Exchange +Input [2]: [i_item_id#20, sum#23] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(31) HashAggregate [codegen id : 3] +Input [2]: [i_item_id#20, sum#23] +Keys [1]: [i_item_id#20] +Functions [1]: [sum(cr_return_quantity#16)] +Aggregate Attributes [1]: [sum(cr_return_quantity#16)#24] +Results [2]: [i_item_id#20 AS item_id#25, sum(cr_return_quantity#16)#24 AS cr_item_qty#26] + +(32) BroadcastExchange +Input [2]: [item_id#25, cr_item_qty#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(33) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#25] Join type: Inner Join condition: None -(21) Project [codegen id : 10] -Output [3]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18] -Input [5]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15, i_item_sk#17, i_item_id#18] - -(22) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#19] +(34) Project [codegen id : 6] +Output [3]: [item_id#13, sr_item_qty#14, cr_item_qty#26] +Input [4]: [item_id#13, sr_item_qty#14, item_id#25, cr_item_qty#26] -(23) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#15] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 10] -Output [2]: [cr_return_quantity#14, i_item_id#18] -Input [4]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18, d_date_sk#19] - -(25) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#14, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(cr_return_quantity#14)] -Aggregate Attributes [1]: [sum#20] -Results [2]: [i_item_id#18, sum#21] - -(26) Exchange -Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#18, sum#21] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(cr_return_quantity#14)] -Aggregate Attributes [1]: [sum(cr_return_quantity#14)#22] -Results [2]: [i_item_id#18 AS item_id#23, sum(cr_return_quantity#14)#22 AS cr_item_qty#24] - -(28) BroadcastExchange -Input [2]: [item_id#23, cr_item_qty#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#11] -Right keys [1]: [item_id#23] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 18] -Output [3]: [item_id#11, sr_item_qty#12, cr_item_qty#24] -Input [4]: [item_id#11, sr_item_qty#12, item_id#23, cr_item_qty#24] - -(31) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] +(35) Scan parquet spark_catalog.default.web_returns +Output [3]: [wr_item_sk#27, wr_return_quantity#28, wr_returned_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(wr_returned_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(wr_returned_date_sk#29), dynamicpruningexpression(wr_returned_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Condition : isnotnull(wr_item_sk#25) - -(33) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#29, i_item_id#30] - -(35) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#25] -Right keys [1]: [i_item_sk#29] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 16] -Output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] -Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#29, i_item_id#30] - -(37) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#31] - -(38) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#27] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 16] -Output [2]: [wr_return_quantity#26, i_item_id#30] -Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30, d_date_sk#31] - -(40) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#26, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(wr_return_quantity#26)] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#30, sum#33] - -(41) Exchange -Input [2]: [i_item_id#30, sum#33] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(42) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#30, sum#33] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(wr_return_quantity#26)] -Aggregate Attributes [1]: [sum(wr_return_quantity#26)#34] -Results [2]: [i_item_id#30 AS item_id#35, sum(wr_return_quantity#26)#34 AS wr_item_qty#36] - -(43) BroadcastExchange -Input [2]: [item_id#35, wr_item_qty#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#11] -Right keys [1]: [item_id#35] +(36) CometFilter +Input [3]: [wr_item_sk#27, wr_return_quantity#28, wr_returned_date_sk#29] +Condition : isnotnull(wr_item_sk#27) + +(37) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#31, i_item_id#32] + +(38) CometBroadcastHashJoin +Left output [3]: [wr_item_sk#27, wr_return_quantity#28, wr_returned_date_sk#29] +Right output [2]: [i_item_sk#31, i_item_id#32] +Arguments: [wr_item_sk#27], [i_item_sk#31], Inner, BuildRight + +(39) CometProject +Input [5]: [wr_item_sk#27, wr_return_quantity#28, wr_returned_date_sk#29, i_item_sk#31, i_item_id#32] +Arguments: [wr_return_quantity#28, wr_returned_date_sk#29, i_item_id#32], [wr_return_quantity#28, wr_returned_date_sk#29, i_item_id#32] + +(40) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#33] + +(41) CometBroadcastHashJoin +Left output [3]: [wr_return_quantity#28, wr_returned_date_sk#29, i_item_id#32] +Right output [1]: [d_date_sk#33] +Arguments: [wr_returned_date_sk#29], [d_date_sk#33], Inner, BuildRight + +(42) CometProject +Input [4]: [wr_return_quantity#28, wr_returned_date_sk#29, i_item_id#32, d_date_sk#33] +Arguments: [wr_return_quantity#28, i_item_id#32], [wr_return_quantity#28, i_item_id#32] + +(43) ColumnarToRow [codegen id : 4] +Input [2]: [wr_return_quantity#28, i_item_id#32] + +(44) HashAggregate [codegen id : 4] +Input [2]: [wr_return_quantity#28, i_item_id#32] +Keys [1]: [i_item_id#32] +Functions [1]: [partial_sum(wr_return_quantity#28)] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#32, sum#35] + +(45) Exchange +Input [2]: [i_item_id#32, sum#35] +Arguments: hashpartitioning(i_item_id#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(46) HashAggregate [codegen id : 5] +Input [2]: [i_item_id#32, sum#35] +Keys [1]: [i_item_id#32] +Functions [1]: [sum(wr_return_quantity#28)] +Aggregate Attributes [1]: [sum(wr_return_quantity#28)#36] +Results [2]: [i_item_id#32 AS item_id#37, sum(wr_return_quantity#28)#36 AS wr_item_qty#38] + +(47) BroadcastExchange +Input [2]: [item_id#37, wr_item_qty#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(48) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [item_id#13] +Right keys [1]: [item_id#37] Join type: Inner Join condition: None -(45) Project [codegen id : 18] -Output [8]: [item_id#11, sr_item_qty#12, (((cast(sr_item_qty#12 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS sr_dev#37, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS cr_dev#38, wr_item_qty#36, (((cast(wr_item_qty#36 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS wr_dev#39, (cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as decimal(20,0)) / 3.0) AS average#40] -Input [5]: [item_id#11, sr_item_qty#12, cr_item_qty#24, item_id#35, wr_item_qty#36] +(49) Project [codegen id : 6] +Output [8]: [item_id#13, sr_item_qty#14, (((cast(sr_item_qty#14 as double) / cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#38) as double)) / 3.0) * 100.0) AS sr_dev#39, cr_item_qty#26, (((cast(cr_item_qty#26 as double) / cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#38) as double)) / 3.0) * 100.0) AS cr_dev#40, wr_item_qty#38, (((cast(wr_item_qty#38 as double) / cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#38) as double)) / 3.0) * 100.0) AS wr_dev#41, (cast(((sr_item_qty#14 + cr_item_qty#26) + wr_item_qty#38) as decimal(20,0)) / 3.0) AS average#42] +Input [5]: [item_id#13, sr_item_qty#14, cr_item_qty#26, item_id#37, wr_item_qty#38] -(46) TakeOrderedAndProject -Input [8]: [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] -Arguments: 100, [item_id#11 ASC NULLS FIRST, sr_item_qty#12 ASC NULLS FIRST], [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] +(50) TakeOrderedAndProject +Input [8]: [item_id#13, sr_item_qty#14, sr_dev#39, cr_item_qty#26, cr_dev#40, wr_item_qty#38, wr_dev#41, average#42] +Arguments: 100, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [item_id#13, sr_item_qty#14, sr_dev#39, cr_item_qty#26, cr_dev#40, wr_item_qty#38, wr_dev#41, average#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * Project (61) - +- * BroadcastHashJoin LeftSemi BuildRight (60) - :- * ColumnarToRow (49) - : +- CometFilter (48) - : +- CometScan parquet spark_catalog.default.date_dim (47) - +- BroadcastExchange (59) - +- * Project (58) - +- * BroadcastHashJoin LeftSemi BuildRight (57) - :- * ColumnarToRow (51) - : +- CometScan parquet spark_catalog.default.date_dim (50) - +- BroadcastExchange (56) - +- * ColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan parquet spark_catalog.default.date_dim (52) - - -(47) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_date#41] +BroadcastExchange (64) ++- * ColumnarToRow (63) + +- CometProject (62) + +- CometBroadcastHashJoin (61) + :- CometFilter (52) + : +- CometScan parquet spark_catalog.default.date_dim (51) + +- CometBroadcastExchange (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometScan parquet spark_catalog.default.date_dim (53) + +- CometBroadcastExchange (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) + + +(51) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter -Input [2]: [d_date_sk#7, d_date#41] +(52) CometFilter +Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(49) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#7, d_date#41] - -(50) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#42, d_week_seq#43] +(53) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#9, d_week_seq#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(51) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#42, d_week_seq#43] - -(52) Scan parquet spark_catalog.default.date_dim +(54) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date#44, d_week_seq#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_date, [2000-06-30,2000-09-27,2000-11-17])] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [2]: [d_date#44, d_week_seq#45] Condition : d_date#44 IN (2000-06-30,2000-09-27,2000-11-17) -(54) CometProject +(56) CometProject Input [2]: [d_date#44, d_week_seq#45] Arguments: [d_week_seq#45], [d_week_seq#45] -(55) ColumnarToRow [codegen id : 1] +(57) CometBroadcastExchange Input [1]: [d_week_seq#45] +Arguments: [d_week_seq#45] -(56) BroadcastExchange -Input [1]: [d_week_seq#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(58) CometBroadcastHashJoin +Left output [2]: [d_date#9, d_week_seq#43] +Right output [1]: [d_week_seq#45] +Arguments: [d_week_seq#43], [d_week_seq#45], LeftSemi, BuildRight -(57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#43] -Right keys [1]: [d_week_seq#45] -Join type: LeftSemi -Join condition: None +(59) CometProject +Input [2]: [d_date#9, d_week_seq#43] +Arguments: [d_date#9], [d_date#9] -(58) Project [codegen id : 2] -Output [1]: [d_date#42] -Input [2]: [d_date#42, d_week_seq#43] +(60) CometBroadcastExchange +Input [1]: [d_date#9] +Arguments: [d_date#9] -(59) BroadcastExchange -Input [1]: [d_date#42] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=8] +(61) CometBroadcastHashJoin +Left output [2]: [d_date_sk#7, d_date#8] +Right output [1]: [d_date#9] +Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight -(60) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#41] -Right keys [1]: [d_date#42] -Join type: LeftSemi -Join condition: None +(62) CometProject +Input [2]: [d_date_sk#7, d_date#8] +Arguments: [d_date_sk#7], [d_date_sk#7] -(61) Project [codegen id : 3] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#41] +(63) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] -(62) BroadcastExchange +(64) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 16 Hosting Expression = cr_returned_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#17 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 31 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 35 Hosting Expression = wr_returned_date_sk#29 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt index a8f1ba3f10..7b7e907eea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - WholeStageCodegen (18) + WholeStageCodegen (6) Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] BroadcastHashJoin [item_id,item_id] Project [item_id,sr_item_qty,cr_item_qty] @@ -7,89 +7,77 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - Project [sr_return_quantity,i_item_id] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [sr_return_quantity,sr_returned_date_sk,i_item_id] - BroadcastHashJoin [sr_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sr_return_quantity,i_item_id] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [sr_item_sk,i_item_sk] CometFilter [sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (3) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date,d_date] CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #3 + CometProject [d_date] + CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange #4 + CometProject [d_week_seq] + CometFilter [d_date] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange #5 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date,d_date] + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date] #3 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (11) + BroadcastExchange #7 + WholeStageCodegen (3) HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (10) + Exchange [i_item_id] #8 + WholeStageCodegen (2) HashAggregate [i_item_id,cr_return_quantity] [sum,sum] - Project [cr_return_quantity,i_item_id] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cr_return_quantity,cr_returned_date_sk,i_item_id] - BroadcastHashJoin [cr_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cr_return_quantity,i_item_id] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [cr_item_sk,i_item_sk] CometFilter [cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #6 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (17) + BroadcastExchange #9 + WholeStageCodegen (5) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] InputAdapter - Exchange [i_item_id] #9 - WholeStageCodegen (16) + Exchange [i_item_id] #10 + WholeStageCodegen (4) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - Project [wr_return_quantity,i_item_id] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Project [wr_return_quantity,wr_returned_date_sk,i_item_id] - BroadcastHashJoin [wr_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [wr_return_quantity,i_item_id] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [wr_item_sk,i_item_sk] CometFilter [wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt index 8dc935d1da..468af40d20 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt @@ -1,41 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- * BroadcastHashJoin Inner BuildLeft (35) - :- BroadcastExchange (30) - : +- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.customer_address (4) - : : : +- BroadcastExchange (14) - : : : +- * ColumnarToRow (13) - : : : +- CometFilter (12) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (11) - : : +- BroadcastExchange (20) - : : +- * ColumnarToRow (19) - : : +- CometFilter (18) - : : +- CometScan parquet spark_catalog.default.household_demographics (17) - : +- BroadcastExchange (27) - : +- * ColumnarToRow (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.income_band (23) - +- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.store_returns (31) +TakeOrderedAndProject (32) ++- * Project (31) + +- * ColumnarToRow (30) + +- CometBroadcastHashJoin (29) + :- CometBroadcastExchange (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.customer_address (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.household_demographics (14) + : +- CometBroadcastExchange (22) + : +- CometProject (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.income_band (19) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.store_returns (26) (1) Scan parquet spark_catalog.default.customer @@ -49,162 +44,142 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [ca_address_sk#7, ca_city#8] Condition : ((isnotnull(ca_city#8) AND (ca_city#8 = Edgewood)) AND isnotnull(ca_address_sk#7)) -(6) CometProject +(5) CometProject Input [2]: [ca_address_sk#7, ca_city#8] Arguments: [ca_address_sk#7], [ca_address_sk#7] -(7) ColumnarToRow [codegen id : 1] -Input [1]: [ca_address_sk#7] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ca_address_sk#7] -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#4] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] +Right output [1]: [ca_address_sk#7] +Arguments: [c_current_addr_sk#4], [ca_address_sk#7], Inner, BuildRight -(10) Project [codegen id : 5] -Output [5]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6] +(8) CometProject Input [7]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6, ca_address_sk#7] +Arguments: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6], [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6] -(11) Scan parquet spark_catalog.default.customer_demographics +(9) Scan parquet spark_catalog.default.customer_demographics Output [1]: [cd_demo_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [1]: [cd_demo_sk#9] Condition : isnotnull(cd_demo_sk#9) -(13) ColumnarToRow [codegen id : 2] -Input [1]: [cd_demo_sk#9] - -(14) BroadcastExchange +(11) CometBroadcastExchange Input [1]: [cd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [cd_demo_sk#9] -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#9] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [5]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6] +Right output [1]: [cd_demo_sk#9] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#9], Inner, BuildRight -(16) Project [codegen id : 5] -Output [5]: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9] +(13) CometProject Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9] +Arguments: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9], [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9] -(17) Scan parquet spark_catalog.default.household_demographics +(14) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#10, hd_income_band_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(18) CometFilter +(15) CometFilter Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] Condition : (isnotnull(hd_demo_sk#10) AND isnotnull(hd_income_band_sk#11)) -(19) ColumnarToRow [codegen id : 3] +(16) CometBroadcastExchange Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] +Arguments: [hd_demo_sk#10, hd_income_band_sk#11] -(20) BroadcastExchange -Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [5]: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9] +Right output [2]: [hd_demo_sk#10, hd_income_band_sk#11] +Arguments: [c_current_hdemo_sk#3], [hd_demo_sk#10], Inner, BuildRight -(22) Project [codegen id : 5] -Output [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11] +(18) CometProject Input [7]: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_demo_sk#10, hd_income_band_sk#11] +Arguments: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11], [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11] -(23) Scan parquet spark_catalog.default.income_band +(19) Scan parquet spark_catalog.default.income_band Output [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] ReadSchema: struct -(24) CometFilter +(20) CometFilter Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] Condition : ((((isnotnull(ib_lower_bound#13) AND isnotnull(ib_upper_bound#14)) AND (ib_lower_bound#13 >= 38128)) AND (ib_upper_bound#14 <= 88128)) AND isnotnull(ib_income_band_sk#12)) -(25) CometProject +(21) CometProject Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] Arguments: [ib_income_band_sk#12], [ib_income_band_sk#12] -(26) ColumnarToRow [codegen id : 4] +(22) CometBroadcastExchange Input [1]: [ib_income_band_sk#12] +Arguments: [ib_income_band_sk#12] -(27) BroadcastExchange -Input [1]: [ib_income_band_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(23) CometBroadcastHashJoin +Left output [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11] +Right output [1]: [ib_income_band_sk#12] +Arguments: [hd_income_band_sk#11], [ib_income_band_sk#12], Inner, BuildRight -(28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [hd_income_band_sk#11] -Right keys [1]: [ib_income_band_sk#12] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 5] -Output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] +(24) CometProject Input [6]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11, ib_income_band_sk#12] +Arguments: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9], [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] -(30) BroadcastExchange +(25) CometBroadcastExchange Input [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [plan_id=5] +Arguments: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] -(31) Scan parquet spark_catalog.default.store_returns +(26) Scan parquet spark_catalog.default.store_returns Output [2]: [sr_cdemo_sk#15, sr_returned_date_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_cdemo_sk)] ReadSchema: struct -(32) CometFilter +(27) CometFilter Input [2]: [sr_cdemo_sk#15, sr_returned_date_sk#16] Condition : isnotnull(sr_cdemo_sk#15) -(33) CometProject +(28) CometProject Input [2]: [sr_cdemo_sk#15, sr_returned_date_sk#16] Arguments: [sr_cdemo_sk#15], [sr_cdemo_sk#15] -(34) ColumnarToRow -Input [1]: [sr_cdemo_sk#15] +(29) CometBroadcastHashJoin +Left output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] +Right output [1]: [sr_cdemo_sk#15] +Arguments: [cd_demo_sk#9], [sr_cdemo_sk#15], Inner, BuildLeft -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cd_demo_sk#9] -Right keys [1]: [sr_cdemo_sk#15] -Join type: Inner -Join condition: None +(30) ColumnarToRow [codegen id : 1] +Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, sr_cdemo_sk#15] -(36) Project [codegen id : 6] +(31) Project [codegen id : 1] Output [3]: [c_customer_id#1 AS customer_id#17, concat(c_last_name#6, , , c_first_name#5) AS customername#18, c_customer_id#1] Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, sr_cdemo_sk#15] -(37) TakeOrderedAndProject +(32) TakeOrderedAndProject Input [3]: [customer_id#17, customername#18, c_customer_id#1] Arguments: 100, [c_customer_id#1 ASC NULLS FIRST], [customer_id#17, customername#18] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt index be3451d29c..fe7f7a2071 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt @@ -1,54 +1,34 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] - WholeStageCodegen (6) + WholeStageCodegen (1) Project [c_customer_id,c_last_name,c_first_name] - BroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] + ColumnarToRow InputAdapter - BroadcastExchange #1 - WholeStageCodegen (5) - Project [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - ColumnarToRow - InputAdapter + CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange #1 + CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] + CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] + CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastExchange #2 + CometProject [ca_address_sk] + CometFilter [ca_city,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange #3 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange #4 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + CometBroadcastExchange #5 + CometProject [ib_income_band_sk] + CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] CometFilter [sr_cdemo_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt index ce1cc22629..c8becd0e9c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt @@ -1,51 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Project (27) - : : : +- * BroadcastHashJoin Inner BuildRight (26) - : : : :- * Project (21) - : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometBroadcastExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.web_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.web_page (10) - : : : : +- BroadcastExchange (19) - : : : : +- * ColumnarToRow (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (16) - : : : +- BroadcastExchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : +- BroadcastExchange (32) - : : +- * ColumnarToRow (31) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometScan parquet spark_catalog.default.customer_address (28) - : +- ReusedExchange (35) - +- BroadcastExchange (41) - +- * ColumnarToRow (40) - +- CometFilter (39) - +- CometScan parquet spark_catalog.default.reason (38) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * ColumnarToRow (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometProject (18) + : : : : +- CometBroadcastHashJoin (17) + : : : : :- CometProject (13) + : : : : : +- CometBroadcastHashJoin (12) + : : : : : :- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometBroadcastExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.web_returns (4) + : : : : : +- CometBroadcastExchange (11) + : : : : : +- CometFilter (10) + : : : : : +- CometScan parquet spark_catalog.default.web_page (9) + : : : : +- CometBroadcastExchange (16) + : : : : +- CometFilter (15) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (14) + : : : +- CometBroadcastExchange (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (19) + : : +- CometBroadcastExchange (27) + : : +- CometProject (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.customer_address (24) + : +- CometBroadcastExchange (33) + : +- CometProject (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.date_dim (30) + +- CometBroadcastExchange (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.reason (36) (1) Scan parquet spark_catalog.default.web_sales @@ -82,224 +80,219 @@ Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_re (7) CometBroadcastHashJoin Left output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] Right output [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner +Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner, BuildLeft (8) CometProject Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] Arguments: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -(9) ColumnarToRow [codegen id : 7] -Input [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(10) Scan parquet spark_catalog.default.web_page +(9) Scan parquet spark_catalog.default.web_page Output [1]: [wp_web_page_sk#18] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [1]: [wp_web_page_sk#18] Condition : isnotnull(wp_web_page_sk#18) -(12) ColumnarToRow [codegen id : 1] -Input [1]: [wp_web_page_sk#18] - -(13) BroadcastExchange +(11) CometBroadcastExchange Input [1]: [wp_web_page_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [wp_web_page_sk#18] -(14) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ws_web_page_sk#2] -Right keys [1]: [wp_web_page_sk#18] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Right output [1]: [wp_web_page_sk#18] +Arguments: [ws_web_page_sk#2], [wp_web_page_sk#18], Inner, BuildRight -(15) Project [codegen id : 7] -Output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +(13) CometProject Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] +Arguments: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -(16) Scan parquet spark_catalog.default.customer_demographics +(14) Scan parquet spark_catalog.default.customer_demographics Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct -(17) CometFilter +(15) CometFilter Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(cd_marital_status#20)) AND isnotnull(cd_education_status#21)) AND ((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree )) OR ((cd_marital_status#20 = S) AND (cd_education_status#21 = College ))) OR ((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree )))) -(18) ColumnarToRow [codegen id : 2] -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] - -(19) BroadcastExchange +(16) CometBroadcastExchange Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -(20) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [wr_refunded_cdemo_sk#10] -Right keys [1]: [cd_demo_sk#19] -Join type: Inner -Join condition: ((((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#20 = S) AND (cd_education_status#21 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) +(17) CometBroadcastHashJoin +Left output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Right output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Arguments: [wr_refunded_cdemo_sk#10], [cd_demo_sk#19], Inner, ((((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#20 = S) AND (cd_education_status#21 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))), BuildRight -(21) Project [codegen id : 7] -Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21] +(18) CometProject Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21] -(22) Scan parquet spark_catalog.default.customer_demographics +(19) Scan parquet spark_catalog.default.customer_demographics Output [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] ReadSchema: struct -(23) CometFilter +(20) CometFilter Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] Condition : ((isnotnull(cd_demo_sk#22) AND isnotnull(cd_marital_status#23)) AND isnotnull(cd_education_status#24)) -(24) ColumnarToRow [codegen id : 3] +(21) CometBroadcastExchange Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] +Arguments: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -(25) BroadcastExchange -Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], input[1, string, false], input[2, string, false]),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 7] -Left keys [3]: [wr_returning_cdemo_sk#12, cd_marital_status#20, cd_education_status#21] -Right keys [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21] +Right output [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] +Arguments: [wr_returning_cdemo_sk#12, cd_marital_status#20, cd_education_status#21], [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24], Inner, BuildRight -(27) Project [codegen id : 7] -Output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +(23) CometProject Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21, cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] +Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -(28) Scan parquet spark_catalog.default.customer_address +(24) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#25, ca_state#26, ca_country#27] 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, [IN,NJ,OH]),In(ca_state, [CT,KY,WI])),In(ca_state, [AR,IA,LA]))] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [3]: [ca_address_sk#25, ca_state#26, ca_country#27] Condition : (((isnotnull(ca_country#27) AND (ca_country#27 = United States)) AND isnotnull(ca_address_sk#25)) AND ((ca_state#26 IN (IN,OH,NJ) OR ca_state#26 IN (WI,CT,KY)) OR ca_state#26 IN (LA,IA,AR))) -(30) CometProject +(26) CometProject Input [3]: [ca_address_sk#25, ca_state#26, ca_country#27] Arguments: [ca_address_sk#25, ca_state#26], [ca_address_sk#25, ca_state#26] -(31) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#25, ca_state#26] - -(32) BroadcastExchange +(27) CometBroadcastExchange Input [2]: [ca_address_sk#25, ca_state#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [ca_address_sk#25, ca_state#26] -(33) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [wr_refunded_addr_sk#11] -Right keys [1]: [ca_address_sk#25] -Join type: Inner -Join condition: ((((ca_state#26 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#26 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#26 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))) +(28) CometBroadcastHashJoin +Left output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Right output [2]: [ca_address_sk#25, ca_state#26] +Arguments: [wr_refunded_addr_sk#11], [ca_address_sk#25], Inner, ((((ca_state#26 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#26 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#26 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))), BuildRight -(34) Project [codegen id : 7] -Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +(29) CometProject Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#25, ca_state#26] +Arguments: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -(35) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#28] +(30) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] +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 + +(31) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : ((isnotnull(d_year#29) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) + +(32) CometProject +Input [2]: [d_date_sk#28, d_year#29] +Arguments: [d_date_sk#28], [d_date_sk#28] + +(33) CometBroadcastExchange +Input [1]: [d_date_sk#28] +Arguments: [d_date_sk#28] -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ws_sold_date_sk#7] -Right keys [1]: [d_date_sk#28] -Join type: Inner -Join condition: None +(34) CometBroadcastHashJoin +Left output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Right output [1]: [d_date_sk#28] +Arguments: [ws_sold_date_sk#7], [d_date_sk#28], Inner, BuildRight -(37) Project [codegen id : 7] -Output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +(35) CometProject Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#28] +Arguments: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -(38) Scan parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#29, r_reason_desc#30] +(36) Scan parquet spark_catalog.default.reason +Output [2]: [r_reason_sk#30, r_reason_desc#31] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] PushedFilters: [IsNotNull(r_reason_sk)] ReadSchema: struct -(39) CometFilter -Input [2]: [r_reason_sk#29, r_reason_desc#30] -Condition : isnotnull(r_reason_sk#29) +(37) CometFilter +Input [2]: [r_reason_sk#30, r_reason_desc#31] +Condition : isnotnull(r_reason_sk#30) -(40) ColumnarToRow [codegen id : 6] -Input [2]: [r_reason_sk#29, r_reason_desc#30] +(38) CometBroadcastExchange +Input [2]: [r_reason_sk#30, r_reason_desc#31] +Arguments: [r_reason_sk#30, r_reason_desc#31] -(41) BroadcastExchange -Input [2]: [r_reason_sk#29, r_reason_desc#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Right output [2]: [r_reason_sk#30, r_reason_desc#31] +Arguments: [wr_reason_sk#13], [r_reason_sk#30], Inner, BuildRight -(42) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [wr_reason_sk#13] -Right keys [1]: [r_reason_sk#29] -Join type: Inner -Join condition: None +(40) CometProject +Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#30, r_reason_desc#31] +Arguments: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31], [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31] -(43) Project [codegen id : 7] -Output [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#30] -Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#29, r_reason_desc#30] +(41) ColumnarToRow [codegen id : 1] +Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31] -(44) HashAggregate [codegen id : 7] -Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#30] -Keys [1]: [r_reason_desc#30] +(42) HashAggregate [codegen id : 1] +Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31] +Keys [1]: [r_reason_desc#31] Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] -Aggregate Attributes [6]: [sum#31, count#32, sum#33, count#34, sum#35, count#36] -Results [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] +Aggregate Attributes [6]: [sum#32, count#33, sum#34, count#35, sum#36, count#37] +Results [7]: [r_reason_desc#31, sum#38, count#39, sum#40, count#41, sum#42, count#43] -(45) Exchange -Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Arguments: hashpartitioning(r_reason_desc#30, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(43) Exchange +Input [7]: [r_reason_desc#31, sum#38, count#39, sum#40, count#41, sum#42, count#43] +Arguments: hashpartitioning(r_reason_desc#31, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(46) HashAggregate [codegen id : 8] -Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Keys [1]: [r_reason_desc#30] +(44) HashAggregate [codegen id : 2] +Input [7]: [r_reason_desc#31, sum#38, count#39, sum#40, count#41, sum#42, count#43] +Keys [1]: [r_reason_desc#31] Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] -Aggregate Attributes [3]: [avg(ws_quantity#4)#43, avg(UnscaledValue(wr_refunded_cash#16))#44, avg(UnscaledValue(wr_fee#15))#45] -Results [4]: [substr(r_reason_desc#30, 1, 20) AS substr(r_reason_desc, 1, 20)#46, avg(ws_quantity#4)#43 AS avg(ws_quantity)#47, cast((avg(UnscaledValue(wr_refunded_cash#16))#44 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#48, cast((avg(UnscaledValue(wr_fee#15))#45 / 100.0) as decimal(11,6)) AS avg(wr_fee)#49] +Aggregate Attributes [3]: [avg(ws_quantity#4)#44, avg(UnscaledValue(wr_refunded_cash#16))#45, avg(UnscaledValue(wr_fee#15))#46] +Results [4]: [substr(r_reason_desc#31, 1, 20) AS substr(r_reason_desc, 1, 20)#47, avg(ws_quantity#4)#44 AS avg(ws_quantity)#48, cast((avg(UnscaledValue(wr_refunded_cash#16))#45 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#49, cast((avg(UnscaledValue(wr_fee#15))#46 / 100.0) as decimal(11,6)) AS avg(wr_fee)#50] -(47) TakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] -Arguments: 100, [substr(r_reason_desc, 1, 20)#46 ASC NULLS FIRST, avg(ws_quantity)#47 ASC NULLS FIRST, avg(wr_refunded_cash)#48 ASC NULLS FIRST, avg(wr_fee)#49 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] +(45) TakeOrderedAndProject +Input [4]: [substr(r_reason_desc, 1, 20)#47, avg(ws_quantity)#48, avg(wr_refunded_cash)#49, avg(wr_fee)#50] +Arguments: 100, [substr(r_reason_desc, 1, 20)#47 ASC NULLS FIRST, avg(ws_quantity)#48 ASC NULLS FIRST, avg(wr_refunded_cash)#49 ASC NULLS FIRST, avg(wr_fee)#50 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#47, avg(ws_quantity)#48, avg(wr_refunded_cash)#49, avg(wr_fee)#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(48) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#50] +(46) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] 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 -(49) CometFilter -Input [2]: [d_date_sk#28, d_year#50] -Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2000)) AND isnotnull(d_date_sk#28)) +(47) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : ((isnotnull(d_year#29) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) -(50) CometProject -Input [2]: [d_date_sk#28, d_year#50] +(48) CometProject +Input [2]: [d_date_sk#28, d_year#29] Arguments: [d_date_sk#28], [d_date_sk#28] -(51) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt index 7c5ee727b7..ad94ef82c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] InputAdapter Exchange [r_reason_desc] #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - BroadcastHashJoin [wr_reason_sk,r_reason_sk] - Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] - Project [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] + CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] CometBroadcastExchange #2 @@ -35,41 +35,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + CometBroadcastExchange #4 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometBroadcastExchange #5 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + CometBroadcastExchange #6 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastExchange #7 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange #8 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #9 + CometFilter [r_reason_sk] + CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt index 5bfd2925d2..1c9688ca28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt @@ -1,25 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * Project (20) - +- Window (19) - +- * Sort (18) - +- Exchange (17) - +- * HashAggregate (16) - +- Exchange (15) - +- * HashAggregate (14) - +- * Expand (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (10) - +- * ColumnarToRow (9) - +- CometFilter (8) - +- CometScan parquet spark_catalog.default.item (7) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * Sort (20) + +- Exchange (19) + +- * HashAggregate (18) + +- Exchange (17) + +- * HashAggregate (16) + +- * ColumnarToRow (15) + +- CometExpand (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.date_dim (3) + +- CometBroadcastExchange (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.item (9) (1) Scan parquet spark_catalog.default.web_sales @@ -34,122 +36,133 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] +Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] -(7) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] +(9) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#7, i_class#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) +(10) CometFilter +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Condition : isnotnull(i_item_sk#7) -(9) ColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#7, i_category#8] +(11) CometBroadcastExchange +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Arguments: [i_item_sk#7, i_class#8, i_category#9] -(10) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#1, ws_net_paid#2] +Right output [3]: [i_item_sk#7, i_class#8, i_category#9] +Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(13) CometProject +Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#8, i_category#9] +Arguments: [ws_net_paid#2, i_category#9, i_class#8], [ws_net_paid#2, i_category#9, i_class#8] -(12) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_category#8, i_class#7] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#7, i_category#8] +(14) CometExpand +Input [3]: [ws_net_paid#2, i_category#9, i_class#8] +Arguments: [[ws_net_paid#2, i_category#9, i_class#8, 0], [ws_net_paid#2, i_category#9, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#10, i_class#11, spark_grouping_id#12] -(13) Expand [codegen id : 3] -Input [3]: [ws_net_paid#2, i_category#8, i_class#7] -Arguments: [[ws_net_paid#2, i_category#8, i_class#7, 0], [ws_net_paid#2, i_category#8, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#9, i_class#10, spark_grouping_id#11] +(15) ColumnarToRow [codegen id : 1] +Input [4]: [ws_net_paid#2, i_category#10, i_class#11, spark_grouping_id#12] -(14) HashAggregate [codegen id : 3] -Input [4]: [ws_net_paid#2, i_category#9, i_class#10, spark_grouping_id#11] -Keys [3]: [i_category#9, i_class#10, spark_grouping_id#11] +(16) HashAggregate [codegen id : 1] +Input [4]: [ws_net_paid#2, i_category#10, i_class#11, spark_grouping_id#12] +Keys [3]: [i_category#10, i_class#11, spark_grouping_id#12] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#12] -Results [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] +Aggregate Attributes [1]: [sum#13] +Results [4]: [i_category#10, i_class#11, spark_grouping_id#12, sum#14] -(15) Exchange -Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] -Arguments: hashpartitioning(i_category#9, i_class#10, spark_grouping_id#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(17) Exchange +Input [4]: [i_category#10, i_class#11, spark_grouping_id#12, sum#14] +Arguments: hashpartitioning(i_category#10, i_class#11, spark_grouping_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(16) HashAggregate [codegen id : 4] -Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] -Keys [3]: [i_category#9, i_class#10, spark_grouping_id#11] +(18) HashAggregate [codegen id : 2] +Input [4]: [i_category#10, i_class#11, spark_grouping_id#12, sum#14] +Keys [3]: [i_category#10, i_class#11, spark_grouping_id#12] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#14] -Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS total_sum#15, i_category#9, i_class#10, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS lochierarchy#16, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS _w0#17, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS _w1#18, CASE WHEN (cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint) = 0) THEN i_category#9 END AS _w2#19] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#15] +Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) AS total_sum#16, i_category#10, i_class#11, (cast((shiftright(spark_grouping_id#12, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#12, 0) & 1) as tinyint)) AS lochierarchy#17, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) AS _w0#18, (cast((shiftright(spark_grouping_id#12, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#12, 0) & 1) as tinyint)) AS _w1#19, CASE WHEN (cast((shiftright(spark_grouping_id#12, 0) & 1) as tinyint) = 0) THEN i_category#10 END AS _w2#20] -(17) Exchange -Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] -Arguments: hashpartitioning(_w1#18, _w2#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(19) Exchange +Input [7]: [total_sum#16, i_category#10, i_class#11, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(18) Sort [codegen id : 5] -Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] -Arguments: [_w1#18 ASC NULLS FIRST, _w2#19 ASC NULLS FIRST, _w0#17 DESC NULLS LAST], false, 0 +(20) Sort [codegen id : 3] +Input [7]: [total_sum#16, i_category#10, i_class#11, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST], false, 0 -(19) Window -Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] -Arguments: [rank(_w0#17) windowspecdefinition(_w1#18, _w2#19, _w0#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#20], [_w1#18, _w2#19], [_w0#17 DESC NULLS LAST] +(21) Window +Input [7]: [total_sum#16, i_category#10, i_class#11, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] -(20) Project [codegen id : 6] -Output [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] -Input [8]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19, rank_within_parent#20] +(22) Project [codegen id : 4] +Output [5]: [total_sum#16, i_category#10, i_class#11, lochierarchy#17, rank_within_parent#21] +Input [8]: [total_sum#16, i_category#10, i_class#11, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] -(21) TakeOrderedAndProject -Input [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] -Arguments: 100, [lochierarchy#16 DESC NULLS LAST, CASE WHEN (lochierarchy#16 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#20 ASC NULLS FIRST], [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] +(23) TakeOrderedAndProject +Input [5]: [total_sum#16, i_category#10, i_class#11, lochierarchy#17, rank_within_parent#21] +Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#11, lochierarchy#17, rank_within_parent#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (26) -+- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) -(22) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#21] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(23) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#21] -Condition : (((isnotnull(d_month_seq#21) AND (d_month_seq#21 >= 1200)) AND (d_month_seq#21 <= 1211)) AND isnotnull(d_date_sk#5)) +(25) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(24) CometProject -Input [2]: [d_date_sk#5, d_month_seq#21] +(26) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(25) ColumnarToRow [codegen id : 1] +(27) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(26) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt index f9db2ce7a4..ea5dce5b69 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (6) + WholeStageCodegen (4) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [_w1,_w2,_w0] InputAdapter Exchange [_w1,_w2] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] InputAdapter Exchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [ws_net_paid,i_category,i_class] - Project [ws_net_paid,i_category,i_class] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -30,12 +30,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #5 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt index a4c9f13ce6..0434066e67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt @@ -1,51 +1,53 @@ == Physical Plan == -* HashAggregate (47) -+- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin LeftAnti BuildRight (43) - :- * BroadcastHashJoin LeftAnti BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer (7) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- BroadcastExchange (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * ColumnarToRow (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.web_sales (30) - : +- ReusedExchange (33) - +- ReusedExchange (36) +* HashAggregate (49) ++- Exchange (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftAnti BuildRight (45) + :- * BroadcastHashJoin LeftAnti BuildRight (31) + : :- * HashAggregate (17) + : : +- Exchange (16) + : : +- * ColumnarToRow (15) + : : +- CometHashAggregate (14) + : : +- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.customer (9) + : +- BroadcastExchange (30) + : +- * HashAggregate (29) + : +- Exchange (28) + : +- * ColumnarToRow (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (44) + +- * HashAggregate (43) + +- Exchange (42) + +- * ColumnarToRow (41) + +- CometHashAggregate (40) + +- CometProject (39) + +- CometBroadcastHashJoin (38) + :- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometFilter (33) + : : +- CometScan parquet spark_catalog.default.web_sales (32) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.store_sales @@ -60,262 +62,263 @@ ReadSchema: struct Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +(3) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#4, d_date#5] +(4) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#2] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] + +(6) CometBroadcastExchange +Input [2]: [d_date_sk#4, d_date#5] +Arguments: [d_date_sk#4, d_date#5] -(6) Project [codegen id : 3] -Output [2]: [ss_customer_sk#1, d_date#5] +(7) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +Right output [2]: [d_date_sk#4, d_date#5] +Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight + +(8) CometProject Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] +Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] -(7) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +(9) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Condition : isnotnull(c_customer_sk#6) - -(9) ColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +(10) CometFilter +Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Condition : isnotnull(c_customer_sk#7) -(10) BroadcastExchange -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [c_customer_sk#7, c_first_name#8, c_last_name#9] -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#6] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#1, d_date#5] +Right output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight -(12) Project [codegen id : 3] -Output [3]: [c_last_name#8, c_first_name#7, d_date#5] -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#7, c_last_name#8] +(13) CometProject +Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [c_last_name#9, c_first_name#8, d_date#5], [c_last_name#9, c_first_name#8, d_date#5] -(13) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +(14) CometHashAggregate +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] -(14) Exchange -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(15) ColumnarToRow [codegen id : 1] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] + +(16) Exchange +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +(17) HashAggregate [codegen id : 6] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] +Results [3]: [c_last_name#9, c_first_name#8, d_date#5] -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +(18) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(17) CometFilter -Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] -Condition : isnotnull(cs_bill_customer_sk#9) - -(18) ColumnarToRow [codegen id : 6] -Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +(19) CometFilter +Input [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] +Condition : isnotnull(cs_bill_customer_sk#10) -(19) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#12, d_date#13] +(20) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#13, d_date#14] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(21) CometBroadcastHashJoin +Left output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] +Right output [2]: [d_date_sk#13, d_date#14] +Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(21) Project [codegen id : 6] -Output [2]: [cs_bill_customer_sk#9, d_date#13] -Input [4]: [cs_bill_customer_sk#9, cs_sold_date_sk#10, d_date_sk#12, d_date#13] +(22) CometProject +Input [4]: [cs_bill_customer_sk#10, cs_sold_date_sk#11, d_date_sk#13, d_date#14] +Arguments: [cs_bill_customer_sk#10, d_date#14], [cs_bill_customer_sk#10, d_date#14] -(22) ReusedExchange [Reuses operator id: 10] -Output [3]: [c_customer_sk#14, c_first_name#15, c_last_name#16] +(23) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_bill_customer_sk#9] -Right keys [1]: [c_customer_sk#14] -Join type: Inner -Join condition: None +(24) CometBroadcastHashJoin +Left output [2]: [cs_bill_customer_sk#10, d_date#14] +Right output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [cs_bill_customer_sk#10], [c_customer_sk#15], Inner, BuildRight -(24) Project [codegen id : 6] -Output [3]: [c_last_name#16, c_first_name#15, d_date#13] -Input [5]: [cs_bill_customer_sk#9, d_date#13, c_customer_sk#14, c_first_name#15, c_last_name#16] +(25) CometProject +Input [5]: [cs_bill_customer_sk#10, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [c_last_name#17, c_first_name#16, d_date#14], [c_last_name#17, c_first_name#16, d_date#14] -(25) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +(26) CometHashAggregate +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(26) Exchange -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(27) ColumnarToRow [codegen id : 2] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -(27) HashAggregate [codegen id : 7] -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +(28) Exchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(29) HashAggregate [codegen id : 3] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#16, c_first_name#15, d_date#13] +Results [3]: [c_last_name#17, c_first_name#16, d_date#14] -(28) BroadcastExchange -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] +(30) BroadcastExchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] -(29) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#16, ), isnull(c_last_name#16), coalesce(c_first_name#15, ), isnull(c_first_name#15), coalesce(d_date#13, 1970-01-01), isnull(d_date#13)] +(31) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] Join type: LeftAnti Join condition: None -(30) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +(32) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#18), dynamicpruningexpression(ws_sold_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] -Condition : isnotnull(ws_bill_customer_sk#17) - -(32) ColumnarToRow [codegen id : 10] -Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +(33) CometFilter +Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Condition : isnotnull(ws_bill_customer_sk#18) -(33) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#20, d_date#21] +(34) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#21, d_date#22] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(35) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Right output [2]: [d_date_sk#21, d_date#22] +Arguments: [ws_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight -(35) Project [codegen id : 10] -Output [2]: [ws_bill_customer_sk#17, d_date#21] -Input [4]: [ws_bill_customer_sk#17, ws_sold_date_sk#18, d_date_sk#20, d_date#21] +(36) CometProject +Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#21, d_date#22] +Arguments: [ws_bill_customer_sk#18, d_date#22], [ws_bill_customer_sk#18, d_date#22] -(36) ReusedExchange [Reuses operator id: 10] -Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] +(37) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_bill_customer_sk#17] -Right keys [1]: [c_customer_sk#22] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, d_date#22] +Right output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [ws_bill_customer_sk#18], [c_customer_sk#23], Inner, BuildRight -(38) Project [codegen id : 10] -Output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Input [5]: [ws_bill_customer_sk#17, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] +(39) CometProject +Input [5]: [ws_bill_customer_sk#18, d_date#22, c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [c_last_name#25, c_first_name#24, d_date#22], [c_last_name#25, c_first_name#24, d_date#22] -(39) HashAggregate [codegen id : 10] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +(40) CometHashAggregate +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(40) Exchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(41) ColumnarToRow [codegen id : 4] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] + +(42) Exchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: hashpartitioning(c_last_name#25, c_first_name#24, d_date#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(41) HashAggregate [codegen id : 11] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +(43) HashAggregate [codegen id : 5] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] +Results [3]: [c_last_name#25, c_first_name#24, d_date#22] -(42) BroadcastExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] +(44) BroadcastExchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] +(45) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#25, ), isnull(c_last_name#25), coalesce(c_first_name#24, ), isnull(c_first_name#24), coalesce(d_date#22, 1970-01-01), isnull(d_date#22)] Join type: LeftAnti Join condition: None -(44) Project [codegen id : 12] +(46) Project [codegen id : 6] Output: [] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(45) HashAggregate [codegen id : 12] +(47) HashAggregate [codegen id : 6] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#25] -Results [1]: [count#26] +Aggregate Attributes [1]: [count#26] +Results [1]: [count#27] -(46) Exchange -Input [1]: [count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(48) Exchange +Input [1]: [count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(47) HashAggregate [codegen id : 13] -Input [1]: [count#26] +(49) HashAggregate [codegen id : 7] +Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#27] -Results [1]: [count(1)#27 AS count(1)#28] +Aggregate Attributes [1]: [count(1)#28] +Results [1]: [count(1)#28 AS count(1)#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(48) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +(50) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) +(51) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) -(50) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +(52) CometProject +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(52) BroadcastExchange +(54) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 +Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt index 315afe6602..cfac83844a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (13) +WholeStageCodegen (7) HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 - WholeStageCodegen (12) + WholeStageCodegen (6) HashAggregate [count,count] Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] @@ -10,14 +10,14 @@ WholeStageCodegen (13) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometProject [ss_customer_sk,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -28,54 +28,48 @@ WholeStageCodegen (13) CometProject [d_date_sk,d_date] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange #4 + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange #5 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (3) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,d_date] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) + BroadcastExchange #8 + WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [c_last_name,c_first_name,d_date] #9 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometProject [ws_bill_customer_sk,d_date] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt index a5f68e5648..3f905ebe90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt @@ -1,186 +1,176 @@ == Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (182) -:- * BroadcastNestedLoopJoin Inner BuildRight (160) -: :- * BroadcastNestedLoopJoin Inner BuildRight (138) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (116) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (94) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (72) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : : : : : :- * HashAggregate (28) -: : : : : : : +- Exchange (27) -: : : : : : : +- * HashAggregate (26) -: : : : : : : +- * Project (25) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (24) -: : : : : : : :- * Project (18) -: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) -: : : : : : : : :- * Project (11) -: : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (10) -: : : : : : : : : :- * ColumnarToRow (4) -: : : : : : : : : : +- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- BroadcastExchange (9) -: : : : : : : : : +- * ColumnarToRow (8) -: : : : : : : : : +- CometProject (7) -: : : : : : : : : +- CometFilter (6) -: : : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (5) -: : : : : : : : +- BroadcastExchange (16) -: : : : : : : : +- * ColumnarToRow (15) -: : : : : : : : +- CometProject (14) -: : : : : : : : +- CometFilter (13) -: : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (12) -: : : : : : : +- BroadcastExchange (23) -: : : : : : : +- * ColumnarToRow (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometFilter (20) -: : : : : : : +- CometScan parquet spark_catalog.default.store (19) -: : : : : : +- BroadcastExchange (49) -: : : : : : +- * HashAggregate (48) -: : : : : : +- Exchange (47) -: : : : : : +- * HashAggregate (46) -: : : : : : +- * Project (45) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (44) -: : : : : : :- * Project (42) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) -: : : : : : : :- * Project (35) -: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) -: : : : : : : : :- * ColumnarToRow (32) -: : : : : : : : : +- CometProject (31) -: : : : : : : : : +- CometFilter (30) -: : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) -: : : : : : : : +- ReusedExchange (33) -: : : : : : : +- BroadcastExchange (40) -: : : : : : : +- * ColumnarToRow (39) -: : : : : : : +- CometProject (38) -: : : : : : : +- CometFilter (37) -: : : : : : : +- CometScan parquet spark_catalog.default.time_dim (36) -: : : : : : +- ReusedExchange (43) -: : : : : +- BroadcastExchange (71) -: : : : : +- * HashAggregate (70) -: : : : : +- Exchange (69) -: : : : : +- * HashAggregate (68) -: : : : : +- * Project (67) -: : : : : +- * BroadcastHashJoin Inner BuildRight (66) -: : : : : :- * Project (64) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (63) -: : : : : : :- * Project (57) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) -: : : : : : : :- * ColumnarToRow (54) -: : : : : : : : +- CometProject (53) -: : : : : : : : +- CometFilter (52) -: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (51) -: : : : : : : +- ReusedExchange (55) -: : : : : : +- BroadcastExchange (62) -: : : : : : +- * ColumnarToRow (61) -: : : : : : +- CometProject (60) -: : : : : : +- CometFilter (59) -: : : : : : +- CometScan parquet spark_catalog.default.time_dim (58) -: : : : : +- ReusedExchange (65) -: : : : +- BroadcastExchange (93) -: : : : +- * HashAggregate (92) -: : : : +- Exchange (91) -: : : : +- * HashAggregate (90) -: : : : +- * Project (89) -: : : : +- * BroadcastHashJoin Inner BuildRight (88) -: : : : :- * Project (86) -: : : : : +- * BroadcastHashJoin Inner BuildRight (85) -: : : : : :- * Project (79) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (78) -: : : : : : :- * ColumnarToRow (76) -: : : : : : : +- CometProject (75) -: : : : : : : +- CometFilter (74) -: : : : : : : +- CometScan parquet spark_catalog.default.store_sales (73) -: : : : : : +- ReusedExchange (77) -: : : : : +- BroadcastExchange (84) -: : : : : +- * ColumnarToRow (83) -: : : : : +- CometProject (82) -: : : : : +- CometFilter (81) -: : : : : +- CometScan parquet spark_catalog.default.time_dim (80) -: : : : +- ReusedExchange (87) -: : : +- BroadcastExchange (115) -: : : +- * HashAggregate (114) -: : : +- Exchange (113) -: : : +- * HashAggregate (112) -: : : +- * Project (111) -: : : +- * BroadcastHashJoin Inner BuildRight (110) -: : : :- * Project (108) -: : : : +- * BroadcastHashJoin Inner BuildRight (107) -: : : : :- * Project (101) -: : : : : +- * BroadcastHashJoin Inner BuildRight (100) -: : : : : :- * ColumnarToRow (98) -: : : : : : +- CometProject (97) -: : : : : : +- CometFilter (96) -: : : : : : +- CometScan parquet spark_catalog.default.store_sales (95) -: : : : : +- ReusedExchange (99) -: : : : +- BroadcastExchange (106) -: : : : +- * ColumnarToRow (105) -: : : : +- CometProject (104) -: : : : +- CometFilter (103) -: : : : +- CometScan parquet spark_catalog.default.time_dim (102) -: : : +- ReusedExchange (109) -: : +- BroadcastExchange (137) -: : +- * HashAggregate (136) -: : +- Exchange (135) -: : +- * HashAggregate (134) -: : +- * Project (133) -: : +- * BroadcastHashJoin Inner BuildRight (132) -: : :- * Project (130) -: : : +- * BroadcastHashJoin Inner BuildRight (129) -: : : :- * Project (123) -: : : : +- * BroadcastHashJoin Inner BuildRight (122) -: : : : :- * ColumnarToRow (120) -: : : : : +- CometProject (119) -: : : : : +- CometFilter (118) -: : : : : +- CometScan parquet spark_catalog.default.store_sales (117) -: : : : +- ReusedExchange (121) -: : : +- BroadcastExchange (128) -: : : +- * ColumnarToRow (127) -: : : +- CometProject (126) -: : : +- CometFilter (125) -: : : +- CometScan parquet spark_catalog.default.time_dim (124) -: : +- ReusedExchange (131) -: +- BroadcastExchange (159) -: +- * HashAggregate (158) -: +- Exchange (157) -: +- * HashAggregate (156) -: +- * Project (155) -: +- * BroadcastHashJoin Inner BuildRight (154) -: :- * Project (152) -: : +- * BroadcastHashJoin Inner BuildRight (151) -: : :- * Project (145) -: : : +- * BroadcastHashJoin Inner BuildRight (144) -: : : :- * ColumnarToRow (142) -: : : : +- CometProject (141) -: : : : +- CometFilter (140) -: : : : +- CometScan parquet spark_catalog.default.store_sales (139) -: : : +- ReusedExchange (143) -: : +- BroadcastExchange (150) -: : +- * ColumnarToRow (149) -: : +- CometProject (148) -: : +- CometFilter (147) -: : +- CometScan parquet spark_catalog.default.time_dim (146) -: +- ReusedExchange (153) -+- BroadcastExchange (181) - +- * HashAggregate (180) - +- Exchange (179) - +- * HashAggregate (178) - +- * Project (177) - +- * BroadcastHashJoin Inner BuildRight (176) - :- * Project (174) - : +- * BroadcastHashJoin Inner BuildRight (173) - : :- * Project (167) - : : +- * BroadcastHashJoin Inner BuildRight (166) - : : :- * ColumnarToRow (164) - : : : +- CometProject (163) - : : : +- CometFilter (162) - : : : +- CometScan parquet spark_catalog.default.store_sales (161) - : : +- ReusedExchange (165) - : +- BroadcastExchange (172) - : +- * ColumnarToRow (171) - : +- CometProject (170) - : +- CometFilter (169) - : +- CometScan parquet spark_catalog.default.time_dim (168) - +- ReusedExchange (175) +* BroadcastNestedLoopJoin Inner BuildRight (172) +:- * BroadcastNestedLoopJoin Inner BuildRight (151) +: :- * BroadcastNestedLoopJoin Inner BuildRight (130) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) +: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) +: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) +: : : : : : :- * HashAggregate (25) +: : : : : : : +- Exchange (24) +: : : : : : : +- * ColumnarToRow (23) +: : : : : : : +- CometHashAggregate (22) +: : : : : : : +- CometProject (21) +: : : : : : : +- CometBroadcastHashJoin (20) +: : : : : : : :- CometProject (15) +: : : : : : : : +- CometBroadcastHashJoin (14) +: : : : : : : : :- CometProject (9) +: : : : : : : : : +- CometBroadcastHashJoin (8) +: : : : : : : : : :- CometProject (3) +: : : : : : : : : : +- CometFilter (2) +: : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : : : : : : +- CometBroadcastExchange (7) +: : : : : : : : : +- CometProject (6) +: : : : : : : : : +- CometFilter (5) +: : : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (4) +: : : : : : : : +- CometBroadcastExchange (13) +: : : : : : : : +- CometProject (12) +: : : : : : : : +- CometFilter (11) +: : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (10) +: : : : : : : +- CometBroadcastExchange (19) +: : : : : : : +- CometProject (18) +: : : : : : : +- CometFilter (17) +: : : : : : : +- CometScan parquet spark_catalog.default.store (16) +: : : : : : +- BroadcastExchange (45) +: : : : : : +- * HashAggregate (44) +: : : : : : +- Exchange (43) +: : : : : : +- * ColumnarToRow (42) +: : : : : : +- CometHashAggregate (41) +: : : : : : +- CometProject (40) +: : : : : : +- CometBroadcastHashJoin (39) +: : : : : : :- CometProject (37) +: : : : : : : +- CometBroadcastHashJoin (36) +: : : : : : : :- CometProject (31) +: : : : : : : : +- CometBroadcastHashJoin (30) +: : : : : : : : :- CometProject (28) +: : : : : : : : : +- CometFilter (27) +: : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (26) +: : : : : : : : +- ReusedExchange (29) +: : : : : : : +- CometBroadcastExchange (35) +: : : : : : : +- CometProject (34) +: : : : : : : +- CometFilter (33) +: : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) +: : : : : : +- ReusedExchange (38) +: : : : : +- BroadcastExchange (66) +: : : : : +- * HashAggregate (65) +: : : : : +- Exchange (64) +: : : : : +- * ColumnarToRow (63) +: : : : : +- CometHashAggregate (62) +: : : : : +- CometProject (61) +: : : : : +- CometBroadcastHashJoin (60) +: : : : : :- CometProject (58) +: : : : : : +- CometBroadcastHashJoin (57) +: : : : : : :- CometProject (52) +: : : : : : : +- CometBroadcastHashJoin (51) +: : : : : : : :- CometProject (49) +: : : : : : : : +- CometFilter (48) +: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (47) +: : : : : : : +- ReusedExchange (50) +: : : : : : +- CometBroadcastExchange (56) +: : : : : : +- CometProject (55) +: : : : : : +- CometFilter (54) +: : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) +: : : : : +- ReusedExchange (59) +: : : : +- BroadcastExchange (87) +: : : : +- * HashAggregate (86) +: : : : +- Exchange (85) +: : : : +- * ColumnarToRow (84) +: : : : +- CometHashAggregate (83) +: : : : +- CometProject (82) +: : : : +- CometBroadcastHashJoin (81) +: : : : :- CometProject (79) +: : : : : +- CometBroadcastHashJoin (78) +: : : : : :- CometProject (73) +: : : : : : +- CometBroadcastHashJoin (72) +: : : : : : :- CometProject (70) +: : : : : : : +- CometFilter (69) +: : : : : : : +- CometScan parquet spark_catalog.default.store_sales (68) +: : : : : : +- ReusedExchange (71) +: : : : : +- CometBroadcastExchange (77) +: : : : : +- CometProject (76) +: : : : : +- CometFilter (75) +: : : : : +- CometScan parquet spark_catalog.default.time_dim (74) +: : : : +- ReusedExchange (80) +: : : +- BroadcastExchange (108) +: : : +- * HashAggregate (107) +: : : +- Exchange (106) +: : : +- * ColumnarToRow (105) +: : : +- CometHashAggregate (104) +: : : +- CometProject (103) +: : : +- CometBroadcastHashJoin (102) +: : : :- CometProject (100) +: : : : +- CometBroadcastHashJoin (99) +: : : : :- CometProject (94) +: : : : : +- CometBroadcastHashJoin (93) +: : : : : :- CometProject (91) +: : : : : : +- CometFilter (90) +: : : : : : +- CometScan parquet spark_catalog.default.store_sales (89) +: : : : : +- ReusedExchange (92) +: : : : +- CometBroadcastExchange (98) +: : : : +- CometProject (97) +: : : : +- CometFilter (96) +: : : : +- CometScan parquet spark_catalog.default.time_dim (95) +: : : +- ReusedExchange (101) +: : +- BroadcastExchange (129) +: : +- * HashAggregate (128) +: : +- Exchange (127) +: : +- * ColumnarToRow (126) +: : +- CometHashAggregate (125) +: : +- CometProject (124) +: : +- CometBroadcastHashJoin (123) +: : :- CometProject (121) +: : : +- CometBroadcastHashJoin (120) +: : : :- CometProject (115) +: : : : +- CometBroadcastHashJoin (114) +: : : : :- CometProject (112) +: : : : : +- CometFilter (111) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (110) +: : : : +- ReusedExchange (113) +: : : +- CometBroadcastExchange (119) +: : : +- CometProject (118) +: : : +- CometFilter (117) +: : : +- CometScan parquet spark_catalog.default.time_dim (116) +: : +- ReusedExchange (122) +: +- BroadcastExchange (150) +: +- * HashAggregate (149) +: +- Exchange (148) +: +- * ColumnarToRow (147) +: +- CometHashAggregate (146) +: +- CometProject (145) +: +- CometBroadcastHashJoin (144) +: :- CometProject (142) +: : +- CometBroadcastHashJoin (141) +: : :- CometProject (136) +: : : +- CometBroadcastHashJoin (135) +: : : :- CometProject (133) +: : : : +- CometFilter (132) +: : : : +- CometScan parquet spark_catalog.default.store_sales (131) +: : : +- ReusedExchange (134) +: : +- CometBroadcastExchange (140) +: : +- CometProject (139) +: : +- CometFilter (138) +: : +- CometScan parquet spark_catalog.default.time_dim (137) +: +- ReusedExchange (143) ++- BroadcastExchange (171) + +- * HashAggregate (170) + +- Exchange (169) + +- * ColumnarToRow (168) + +- CometHashAggregate (167) + +- CometProject (166) + +- CometBroadcastHashJoin (165) + :- CometProject (163) + : +- CometBroadcastHashJoin (162) + : :- CometProject (157) + : : +- CometBroadcastHashJoin (156) + : : :- CometProject (154) + : : : +- CometFilter (153) + : : : +- CometScan parquet spark_catalog.default.store_sales (152) + : : +- ReusedExchange (155) + : +- CometBroadcastExchange (161) + : +- CometProject (160) + : +- CometFilter (159) + : +- CometScan parquet spark_catalog.default.time_dim (158) + +- ReusedExchange (164) (1) Scan parquet spark_catalog.default.store_sales @@ -198,834 +188,756 @@ Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isn Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -(4) ColumnarToRow [codegen id : 4] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(5) Scan parquet spark_catalog.default.household_demographics +(4) Scan parquet spark_catalog.default.household_demographics Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) -(7) CometProject +(6) CometProject Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] Arguments: [hd_demo_sk#5], [hd_demo_sk#5] -(8) ColumnarToRow [codegen id : 1] +(7) CometBroadcastExchange Input [1]: [hd_demo_sk#5] +Arguments: [hd_demo_sk#5] -(9) BroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(8) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Right output [1]: [hd_demo_sk#5] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight -(10) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 4] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +(9) CometProject Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] -(12) Scan parquet spark_catalog.default.time_dim +(10) Scan parquet spark_catalog.default.time_dim Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(13) CometFilter +(11) CometFilter Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) -(14) CometProject +(12) CometProject Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] Arguments: [t_time_sk#8], [t_time_sk#8] -(15) ColumnarToRow [codegen id : 2] +(13) CometBroadcastExchange Input [1]: [t_time_sk#8] +Arguments: [t_time_sk#8] -(16) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(14) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Right output [1]: [t_time_sk#8] +Arguments: [ss_sold_time_sk#1], [t_time_sk#8], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [1]: [ss_store_sk#3] +(15) CometProject Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] +Arguments: [ss_store_sk#3], [ss_store_sk#3] -(19) Scan parquet spark_catalog.default.store +(16) Scan parquet spark_catalog.default.store Output [2]: [s_store_sk#11, s_store_name#12] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] ReadSchema: struct -(20) CometFilter +(17) CometFilter Input [2]: [s_store_sk#11, s_store_name#12] Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) -(21) CometProject +(18) CometProject Input [2]: [s_store_sk#11, s_store_name#12] Arguments: [s_store_sk#11], [s_store_sk#11] -(22) ColumnarToRow [codegen id : 3] +(19) CometBroadcastExchange Input [1]: [s_store_sk#11] +Arguments: [s_store_sk#11] -(23) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#3] +Right output [1]: [s_store_sk#11] +Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight -(25) Project [codegen id : 4] -Output: [] +(21) CometProject Input [2]: [ss_store_sk#3, s_store_sk#11] -(26) HashAggregate [codegen id : 4] +(22) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#13] -Results [1]: [count#14] -(27) Exchange -Input [1]: [count#14] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(23) ColumnarToRow [codegen id : 1] +Input [1]: [count#13] + +(24) Exchange +Input [1]: [count#13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 40] -Input [1]: [count#14] +(25) HashAggregate [codegen id : 16] +Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#15] -Results [1]: [count(1)#15 AS h8_30_to_9#16] +Aggregate Attributes [1]: [count(1)#14] +Results [1]: [count(1)#14 AS h8_30_to_9#15] -(29) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] +(26) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_hdemo_sk#18) AND isnotnull(ss_sold_time_sk#17)) AND isnotnull(ss_store_sk#19)) +(27) CometFilter +Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] +Condition : ((isnotnull(ss_hdemo_sk#17) AND isnotnull(ss_sold_time_sk#16)) AND isnotnull(ss_store_sk#18)) -(31) CometProject -Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] -Arguments: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19], [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] +(28) CometProject +Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] +Arguments: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18], [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18] -(32) ColumnarToRow [codegen id : 8] -Input [3]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] +(29) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#20] -(33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#21] +(30) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18] +Right output [1]: [hd_demo_sk#20] +Arguments: [ss_hdemo_sk#17], [hd_demo_sk#20], Inner, BuildRight -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#18] -Right keys [1]: [hd_demo_sk#21] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 8] -Output [2]: [ss_sold_time_sk#17, ss_store_sk#19] -Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, hd_demo_sk#21] +(31) CometProject +Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, hd_demo_sk#20] +Arguments: [ss_sold_time_sk#16, ss_store_sk#18], [ss_sold_time_sk#16, ss_store_sk#18] -(36) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#22, t_hour#23, t_minute#24] +(32) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#21, t_hour#22, t_minute#23] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(37) CometFilter -Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] -Condition : ((((isnotnull(t_hour#23) AND isnotnull(t_minute#24)) AND (t_hour#23 = 9)) AND (t_minute#24 < 30)) AND isnotnull(t_time_sk#22)) +(33) CometFilter +Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Condition : ((((isnotnull(t_hour#22) AND isnotnull(t_minute#23)) AND (t_hour#22 = 9)) AND (t_minute#23 < 30)) AND isnotnull(t_time_sk#21)) -(38) CometProject -Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] -Arguments: [t_time_sk#22], [t_time_sk#22] +(34) CometProject +Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Arguments: [t_time_sk#21], [t_time_sk#21] -(39) ColumnarToRow [codegen id : 6] -Input [1]: [t_time_sk#22] +(35) CometBroadcastExchange +Input [1]: [t_time_sk#21] +Arguments: [t_time_sk#21] -(40) BroadcastExchange -Input [1]: [t_time_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#16, ss_store_sk#18] +Right output [1]: [t_time_sk#21] +Arguments: [ss_sold_time_sk#16], [t_time_sk#21], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#17] -Right keys [1]: [t_time_sk#22] -Join type: Inner -Join condition: None +(37) CometProject +Input [3]: [ss_sold_time_sk#16, ss_store_sk#18, t_time_sk#21] +Arguments: [ss_store_sk#18], [ss_store_sk#18] -(42) Project [codegen id : 8] -Output [1]: [ss_store_sk#19] -Input [3]: [ss_sold_time_sk#17, ss_store_sk#19, t_time_sk#22] +(38) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#24] -(43) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#25] +(39) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#18] +Right output [1]: [s_store_sk#24] +Arguments: [ss_store_sk#18], [s_store_sk#24], Inner, BuildRight -(44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#19] -Right keys [1]: [s_store_sk#25] -Join type: Inner -Join condition: None +(40) CometProject +Input [2]: [ss_store_sk#18, s_store_sk#24] -(45) Project [codegen id : 8] -Output: [] -Input [2]: [ss_store_sk#19, s_store_sk#25] - -(46) HashAggregate [codegen id : 8] +(41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#26] -Results [1]: [count#27] -(47) Exchange -Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(42) ColumnarToRow [codegen id : 2] +Input [1]: [count#25] -(48) HashAggregate [codegen id : 9] -Input [1]: [count#27] +(43) Exchange +Input [1]: [count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] + +(44) HashAggregate [codegen id : 3] +Input [1]: [count#25] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#28] -Results [1]: [count(1)#28 AS h9_to_9_30#29] +Aggregate Attributes [1]: [count(1)#26] +Results [1]: [count(1)#26 AS h9_to_9_30#27] -(49) BroadcastExchange -Input [1]: [h9_to_9_30#29] -Arguments: IdentityBroadcastMode, [plan_id=7] +(45) BroadcastExchange +Input [1]: [h9_to_9_30#27] +Arguments: IdentityBroadcastMode, [plan_id=3] -(50) BroadcastNestedLoopJoin [codegen id : 40] +(46) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(51) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] +(47) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(52) CometFilter -Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_hdemo_sk#31) AND isnotnull(ss_sold_time_sk#30)) AND isnotnull(ss_store_sk#32)) +(48) CometFilter +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] +Condition : ((isnotnull(ss_hdemo_sk#29) AND isnotnull(ss_sold_time_sk#28)) AND isnotnull(ss_store_sk#30)) -(53) CometProject -Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] -Arguments: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32], [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] +(49) CometProject +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] +Arguments: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30], [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30] -(54) ColumnarToRow [codegen id : 13] -Input [3]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] +(50) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#32] -(55) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#34] - -(56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#31] -Right keys [1]: [hd_demo_sk#34] -Join type: Inner -Join condition: None +(51) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30] +Right output [1]: [hd_demo_sk#32] +Arguments: [ss_hdemo_sk#29], [hd_demo_sk#32], Inner, BuildRight -(57) Project [codegen id : 13] -Output [2]: [ss_sold_time_sk#30, ss_store_sk#32] -Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, hd_demo_sk#34] +(52) CometProject +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, hd_demo_sk#32] +Arguments: [ss_sold_time_sk#28, ss_store_sk#30], [ss_sold_time_sk#28, ss_store_sk#30] -(58) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#35, t_hour#36, t_minute#37] +(53) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#33, t_hour#34, t_minute#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(59) CometFilter -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Condition : ((((isnotnull(t_hour#36) AND isnotnull(t_minute#37)) AND (t_hour#36 = 9)) AND (t_minute#37 >= 30)) AND isnotnull(t_time_sk#35)) +(54) CometFilter +Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Condition : ((((isnotnull(t_hour#34) AND isnotnull(t_minute#35)) AND (t_hour#34 = 9)) AND (t_minute#35 >= 30)) AND isnotnull(t_time_sk#33)) -(60) CometProject -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [t_time_sk#35], [t_time_sk#35] +(55) CometProject +Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Arguments: [t_time_sk#33], [t_time_sk#33] -(61) ColumnarToRow [codegen id : 11] -Input [1]: [t_time_sk#35] +(56) CometBroadcastExchange +Input [1]: [t_time_sk#33] +Arguments: [t_time_sk#33] -(62) BroadcastExchange -Input [1]: [t_time_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +(57) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#28, ss_store_sk#30] +Right output [1]: [t_time_sk#33] +Arguments: [ss_sold_time_sk#28], [t_time_sk#33], Inner, BuildRight -(63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#30] -Right keys [1]: [t_time_sk#35] -Join type: Inner -Join condition: None - -(64) Project [codegen id : 13] -Output [1]: [ss_store_sk#32] -Input [3]: [ss_sold_time_sk#30, ss_store_sk#32, t_time_sk#35] +(58) CometProject +Input [3]: [ss_sold_time_sk#28, ss_store_sk#30, t_time_sk#33] +Arguments: [ss_store_sk#30], [ss_store_sk#30] -(65) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#38] +(59) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#36] -(66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#32] -Right keys [1]: [s_store_sk#38] -Join type: Inner -Join condition: None +(60) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#30] +Right output [1]: [s_store_sk#36] +Arguments: [ss_store_sk#30], [s_store_sk#36], Inner, BuildRight -(67) Project [codegen id : 13] -Output: [] -Input [2]: [ss_store_sk#32, s_store_sk#38] +(61) CometProject +Input [2]: [ss_store_sk#30, s_store_sk#36] -(68) HashAggregate [codegen id : 13] +(62) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#39] -Results [1]: [count#40] -(69) Exchange -Input [1]: [count#40] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +(63) ColumnarToRow [codegen id : 4] +Input [1]: [count#37] -(70) HashAggregate [codegen id : 14] -Input [1]: [count#40] +(64) Exchange +Input [1]: [count#37] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(65) HashAggregate [codegen id : 5] +Input [1]: [count#37] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#41] -Results [1]: [count(1)#41 AS h9_30_to_10#42] +Aggregate Attributes [1]: [count(1)#38] +Results [1]: [count(1)#38 AS h9_30_to_10#39] -(71) BroadcastExchange -Input [1]: [h9_30_to_10#42] -Arguments: IdentityBroadcastMode, [plan_id=10] +(66) BroadcastExchange +Input [1]: [h9_30_to_10#39] +Arguments: IdentityBroadcastMode, [plan_id=5] -(72) BroadcastNestedLoopJoin [codegen id : 40] +(67) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(73) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] +(68) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(74) CometFilter -Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_hdemo_sk#44) AND isnotnull(ss_sold_time_sk#43)) AND isnotnull(ss_store_sk#45)) - -(75) CometProject -Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] -Arguments: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45], [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] +(69) CometFilter +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] +Condition : ((isnotnull(ss_hdemo_sk#41) AND isnotnull(ss_sold_time_sk#40)) AND isnotnull(ss_store_sk#42)) -(76) ColumnarToRow [codegen id : 18] -Input [3]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] +(70) CometProject +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] +Arguments: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42], [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42] -(77) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#47] +(71) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#44] -(78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#44] -Right keys [1]: [hd_demo_sk#47] -Join type: Inner -Join condition: None +(72) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42] +Right output [1]: [hd_demo_sk#44] +Arguments: [ss_hdemo_sk#41], [hd_demo_sk#44], Inner, BuildRight -(79) Project [codegen id : 18] -Output [2]: [ss_sold_time_sk#43, ss_store_sk#45] -Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, hd_demo_sk#47] +(73) CometProject +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, hd_demo_sk#44] +Arguments: [ss_sold_time_sk#40, ss_store_sk#42], [ss_sold_time_sk#40, ss_store_sk#42] -(80) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#48, t_hour#49, t_minute#50] +(74) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#45, t_hour#46, t_minute#47] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(81) CometFilter -Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] -Condition : ((((isnotnull(t_hour#49) AND isnotnull(t_minute#50)) AND (t_hour#49 = 10)) AND (t_minute#50 < 30)) AND isnotnull(t_time_sk#48)) - -(82) CometProject -Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] -Arguments: [t_time_sk#48], [t_time_sk#48] +(75) CometFilter +Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Condition : ((((isnotnull(t_hour#46) AND isnotnull(t_minute#47)) AND (t_hour#46 = 10)) AND (t_minute#47 < 30)) AND isnotnull(t_time_sk#45)) -(83) ColumnarToRow [codegen id : 16] -Input [1]: [t_time_sk#48] +(76) CometProject +Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Arguments: [t_time_sk#45], [t_time_sk#45] -(84) BroadcastExchange -Input [1]: [t_time_sk#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +(77) CometBroadcastExchange +Input [1]: [t_time_sk#45] +Arguments: [t_time_sk#45] -(85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#43] -Right keys [1]: [t_time_sk#48] -Join type: Inner -Join condition: None +(78) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#40, ss_store_sk#42] +Right output [1]: [t_time_sk#45] +Arguments: [ss_sold_time_sk#40], [t_time_sk#45], Inner, BuildRight -(86) Project [codegen id : 18] -Output [1]: [ss_store_sk#45] -Input [3]: [ss_sold_time_sk#43, ss_store_sk#45, t_time_sk#48] +(79) CometProject +Input [3]: [ss_sold_time_sk#40, ss_store_sk#42, t_time_sk#45] +Arguments: [ss_store_sk#42], [ss_store_sk#42] -(87) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#51] +(80) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#48] -(88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#45] -Right keys [1]: [s_store_sk#51] -Join type: Inner -Join condition: None +(81) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#42] +Right output [1]: [s_store_sk#48] +Arguments: [ss_store_sk#42], [s_store_sk#48], Inner, BuildRight -(89) Project [codegen id : 18] -Output: [] -Input [2]: [ss_store_sk#45, s_store_sk#51] +(82) CometProject +Input [2]: [ss_store_sk#42, s_store_sk#48] -(90) HashAggregate [codegen id : 18] +(83) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#52] -Results [1]: [count#53] -(91) Exchange -Input [1]: [count#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(84) ColumnarToRow [codegen id : 6] +Input [1]: [count#49] -(92) HashAggregate [codegen id : 19] -Input [1]: [count#53] +(85) Exchange +Input [1]: [count#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] + +(86) HashAggregate [codegen id : 7] +Input [1]: [count#49] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#54] -Results [1]: [count(1)#54 AS h10_to_10_30#55] +Aggregate Attributes [1]: [count(1)#50] +Results [1]: [count(1)#50 AS h10_to_10_30#51] -(93) BroadcastExchange -Input [1]: [h10_to_10_30#55] -Arguments: IdentityBroadcastMode, [plan_id=13] +(87) BroadcastExchange +Input [1]: [h10_to_10_30#51] +Arguments: IdentityBroadcastMode, [plan_id=7] -(94) BroadcastNestedLoopJoin [codegen id : 40] +(88) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(95) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] +(89) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(96) CometFilter -Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_hdemo_sk#57) AND isnotnull(ss_sold_time_sk#56)) AND isnotnull(ss_store_sk#58)) +(90) CometFilter +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] +Condition : ((isnotnull(ss_hdemo_sk#53) AND isnotnull(ss_sold_time_sk#52)) AND isnotnull(ss_store_sk#54)) -(97) CometProject -Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] -Arguments: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58], [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] +(91) CometProject +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] +Arguments: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54], [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54] -(98) ColumnarToRow [codegen id : 23] -Input [3]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] +(92) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#56] -(99) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#60] +(93) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54] +Right output [1]: [hd_demo_sk#56] +Arguments: [ss_hdemo_sk#53], [hd_demo_sk#56], Inner, BuildRight -(100) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#57] -Right keys [1]: [hd_demo_sk#60] -Join type: Inner -Join condition: None - -(101) Project [codegen id : 23] -Output [2]: [ss_sold_time_sk#56, ss_store_sk#58] -Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, hd_demo_sk#60] +(94) CometProject +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, hd_demo_sk#56] +Arguments: [ss_sold_time_sk#52, ss_store_sk#54], [ss_sold_time_sk#52, ss_store_sk#54] -(102) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#61, t_hour#62, t_minute#63] +(95) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(103) CometFilter -Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] -Condition : ((((isnotnull(t_hour#62) AND isnotnull(t_minute#63)) AND (t_hour#62 = 10)) AND (t_minute#63 >= 30)) AND isnotnull(t_time_sk#61)) +(96) CometFilter +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Condition : ((((isnotnull(t_hour#58) AND isnotnull(t_minute#59)) AND (t_hour#58 = 10)) AND (t_minute#59 >= 30)) AND isnotnull(t_time_sk#57)) -(104) CometProject -Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] -Arguments: [t_time_sk#61], [t_time_sk#61] +(97) CometProject +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Arguments: [t_time_sk#57], [t_time_sk#57] -(105) ColumnarToRow [codegen id : 21] -Input [1]: [t_time_sk#61] +(98) CometBroadcastExchange +Input [1]: [t_time_sk#57] +Arguments: [t_time_sk#57] -(106) BroadcastExchange -Input [1]: [t_time_sk#61] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(99) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#52, ss_store_sk#54] +Right output [1]: [t_time_sk#57] +Arguments: [ss_sold_time_sk#52], [t_time_sk#57], Inner, BuildRight -(107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#56] -Right keys [1]: [t_time_sk#61] -Join type: Inner -Join condition: None +(100) CometProject +Input [3]: [ss_sold_time_sk#52, ss_store_sk#54, t_time_sk#57] +Arguments: [ss_store_sk#54], [ss_store_sk#54] -(108) Project [codegen id : 23] -Output [1]: [ss_store_sk#58] -Input [3]: [ss_sold_time_sk#56, ss_store_sk#58, t_time_sk#61] +(101) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#60] -(109) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#64] +(102) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#54] +Right output [1]: [s_store_sk#60] +Arguments: [ss_store_sk#54], [s_store_sk#60], Inner, BuildRight -(110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#58] -Right keys [1]: [s_store_sk#64] -Join type: Inner -Join condition: None - -(111) Project [codegen id : 23] -Output: [] -Input [2]: [ss_store_sk#58, s_store_sk#64] +(103) CometProject +Input [2]: [ss_store_sk#54, s_store_sk#60] -(112) HashAggregate [codegen id : 23] +(104) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#65] -Results [1]: [count#66] -(113) Exchange -Input [1]: [count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] +(105) ColumnarToRow [codegen id : 8] +Input [1]: [count#61] -(114) HashAggregate [codegen id : 24] -Input [1]: [count#66] +(106) Exchange +Input [1]: [count#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] + +(107) HashAggregate [codegen id : 9] +Input [1]: [count#61] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#67] -Results [1]: [count(1)#67 AS h10_30_to_11#68] +Aggregate Attributes [1]: [count(1)#62] +Results [1]: [count(1)#62 AS h10_30_to_11#63] -(115) BroadcastExchange -Input [1]: [h10_30_to_11#68] -Arguments: IdentityBroadcastMode, [plan_id=16] +(108) BroadcastExchange +Input [1]: [h10_30_to_11#63] +Arguments: IdentityBroadcastMode, [plan_id=9] -(116) BroadcastNestedLoopJoin [codegen id : 40] +(109) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(117) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +(110) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(118) CometFilter -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) - -(119) CometProject -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] -Arguments: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71], [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +(111) CometFilter +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] +Condition : ((isnotnull(ss_hdemo_sk#65) AND isnotnull(ss_sold_time_sk#64)) AND isnotnull(ss_store_sk#66)) -(120) ColumnarToRow [codegen id : 28] -Input [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +(112) CometProject +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] +Arguments: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66], [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66] -(121) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#73] +(113) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#68] -(122) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#70] -Right keys [1]: [hd_demo_sk#73] -Join type: Inner -Join condition: None +(114) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66] +Right output [1]: [hd_demo_sk#68] +Arguments: [ss_hdemo_sk#65], [hd_demo_sk#68], Inner, BuildRight -(123) Project [codegen id : 28] -Output [2]: [ss_sold_time_sk#69, ss_store_sk#71] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, hd_demo_sk#73] +(115) CometProject +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, hd_demo_sk#68] +Arguments: [ss_sold_time_sk#64, ss_store_sk#66], [ss_sold_time_sk#64, ss_store_sk#66] -(124) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#74, t_hour#75, t_minute#76] +(116) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#69, t_hour#70, t_minute#71] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(125) CometFilter -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] -Condition : ((((isnotnull(t_hour#75) AND isnotnull(t_minute#76)) AND (t_hour#75 = 11)) AND (t_minute#76 < 30)) AND isnotnull(t_time_sk#74)) - -(126) CometProject -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] -Arguments: [t_time_sk#74], [t_time_sk#74] +(117) CometFilter +Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Condition : ((((isnotnull(t_hour#70) AND isnotnull(t_minute#71)) AND (t_hour#70 = 11)) AND (t_minute#71 < 30)) AND isnotnull(t_time_sk#69)) -(127) ColumnarToRow [codegen id : 26] -Input [1]: [t_time_sk#74] +(118) CometProject +Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Arguments: [t_time_sk#69], [t_time_sk#69] -(128) BroadcastExchange -Input [1]: [t_time_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] +(119) CometBroadcastExchange +Input [1]: [t_time_sk#69] +Arguments: [t_time_sk#69] -(129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#69] -Right keys [1]: [t_time_sk#74] -Join type: Inner -Join condition: None +(120) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#64, ss_store_sk#66] +Right output [1]: [t_time_sk#69] +Arguments: [ss_sold_time_sk#64], [t_time_sk#69], Inner, BuildRight -(130) Project [codegen id : 28] -Output [1]: [ss_store_sk#71] -Input [3]: [ss_sold_time_sk#69, ss_store_sk#71, t_time_sk#74] +(121) CometProject +Input [3]: [ss_sold_time_sk#64, ss_store_sk#66, t_time_sk#69] +Arguments: [ss_store_sk#66], [ss_store_sk#66] -(131) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#77] +(122) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#72] -(132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#71] -Right keys [1]: [s_store_sk#77] -Join type: Inner -Join condition: None +(123) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#66] +Right output [1]: [s_store_sk#72] +Arguments: [ss_store_sk#66], [s_store_sk#72], Inner, BuildRight -(133) Project [codegen id : 28] -Output: [] -Input [2]: [ss_store_sk#71, s_store_sk#77] +(124) CometProject +Input [2]: [ss_store_sk#66, s_store_sk#72] -(134) HashAggregate [codegen id : 28] +(125) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#78] -Results [1]: [count#79] -(135) Exchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] +(126) ColumnarToRow [codegen id : 10] +Input [1]: [count#73] -(136) HashAggregate [codegen id : 29] -Input [1]: [count#79] +(127) Exchange +Input [1]: [count#73] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] + +(128) HashAggregate [codegen id : 11] +Input [1]: [count#73] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#80] -Results [1]: [count(1)#80 AS h11_to_11_30#81] +Aggregate Attributes [1]: [count(1)#74] +Results [1]: [count(1)#74 AS h11_to_11_30#75] -(137) BroadcastExchange -Input [1]: [h11_to_11_30#81] -Arguments: IdentityBroadcastMode, [plan_id=19] +(129) BroadcastExchange +Input [1]: [h11_to_11_30#75] +Arguments: IdentityBroadcastMode, [plan_id=11] -(138) BroadcastNestedLoopJoin [codegen id : 40] +(130) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(139) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] +(131) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(140) CometFilter -Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] -Condition : ((isnotnull(ss_hdemo_sk#83) AND isnotnull(ss_sold_time_sk#82)) AND isnotnull(ss_store_sk#84)) - -(141) CometProject -Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] -Arguments: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84], [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] +(132) CometFilter +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] +Condition : ((isnotnull(ss_hdemo_sk#77) AND isnotnull(ss_sold_time_sk#76)) AND isnotnull(ss_store_sk#78)) -(142) ColumnarToRow [codegen id : 33] -Input [3]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] +(133) CometProject +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] +Arguments: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78], [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] -(143) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#86] +(134) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#80] -(144) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#83] -Right keys [1]: [hd_demo_sk#86] -Join type: Inner -Join condition: None +(135) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] +Right output [1]: [hd_demo_sk#80] +Arguments: [ss_hdemo_sk#77], [hd_demo_sk#80], Inner, BuildRight -(145) Project [codegen id : 33] -Output [2]: [ss_sold_time_sk#82, ss_store_sk#84] -Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, hd_demo_sk#86] +(136) CometProject +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, hd_demo_sk#80] +Arguments: [ss_sold_time_sk#76, ss_store_sk#78], [ss_sold_time_sk#76, ss_store_sk#78] -(146) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#87, t_hour#88, t_minute#89] +(137) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#81, t_hour#82, t_minute#83] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(147) CometFilter -Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] -Condition : ((((isnotnull(t_hour#88) AND isnotnull(t_minute#89)) AND (t_hour#88 = 11)) AND (t_minute#89 >= 30)) AND isnotnull(t_time_sk#87)) +(138) CometFilter +Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Condition : ((((isnotnull(t_hour#82) AND isnotnull(t_minute#83)) AND (t_hour#82 = 11)) AND (t_minute#83 >= 30)) AND isnotnull(t_time_sk#81)) -(148) CometProject -Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] -Arguments: [t_time_sk#87], [t_time_sk#87] +(139) CometProject +Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Arguments: [t_time_sk#81], [t_time_sk#81] -(149) ColumnarToRow [codegen id : 31] -Input [1]: [t_time_sk#87] +(140) CometBroadcastExchange +Input [1]: [t_time_sk#81] +Arguments: [t_time_sk#81] -(150) BroadcastExchange -Input [1]: [t_time_sk#87] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] +(141) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#76, ss_store_sk#78] +Right output [1]: [t_time_sk#81] +Arguments: [ss_sold_time_sk#76], [t_time_sk#81], Inner, BuildRight -(151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#82] -Right keys [1]: [t_time_sk#87] -Join type: Inner -Join condition: None +(142) CometProject +Input [3]: [ss_sold_time_sk#76, ss_store_sk#78, t_time_sk#81] +Arguments: [ss_store_sk#78], [ss_store_sk#78] -(152) Project [codegen id : 33] -Output [1]: [ss_store_sk#84] -Input [3]: [ss_sold_time_sk#82, ss_store_sk#84, t_time_sk#87] +(143) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#84] -(153) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#90] +(144) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#78] +Right output [1]: [s_store_sk#84] +Arguments: [ss_store_sk#78], [s_store_sk#84], Inner, BuildRight -(154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#84] -Right keys [1]: [s_store_sk#90] -Join type: Inner -Join condition: None - -(155) Project [codegen id : 33] -Output: [] -Input [2]: [ss_store_sk#84, s_store_sk#90] +(145) CometProject +Input [2]: [ss_store_sk#78, s_store_sk#84] -(156) HashAggregate [codegen id : 33] +(146) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#91] -Results [1]: [count#92] -(157) Exchange -Input [1]: [count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=21] +(147) ColumnarToRow [codegen id : 12] +Input [1]: [count#85] -(158) HashAggregate [codegen id : 34] -Input [1]: [count#92] +(148) Exchange +Input [1]: [count#85] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] + +(149) HashAggregate [codegen id : 13] +Input [1]: [count#85] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#93] -Results [1]: [count(1)#93 AS h11_30_to_12#94] +Aggregate Attributes [1]: [count(1)#86] +Results [1]: [count(1)#86 AS h11_30_to_12#87] -(159) BroadcastExchange -Input [1]: [h11_30_to_12#94] -Arguments: IdentityBroadcastMode, [plan_id=22] +(150) BroadcastExchange +Input [1]: [h11_30_to_12#87] +Arguments: IdentityBroadcastMode, [plan_id=13] -(160) BroadcastNestedLoopJoin [codegen id : 40] +(151) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(161) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] +(152) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(162) CometFilter -Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] -Condition : ((isnotnull(ss_hdemo_sk#96) AND isnotnull(ss_sold_time_sk#95)) AND isnotnull(ss_store_sk#97)) - -(163) CometProject -Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] -Arguments: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97], [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] +(153) CometFilter +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] +Condition : ((isnotnull(ss_hdemo_sk#89) AND isnotnull(ss_sold_time_sk#88)) AND isnotnull(ss_store_sk#90)) -(164) ColumnarToRow [codegen id : 38] -Input [3]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] +(154) CometProject +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] +Arguments: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90], [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90] -(165) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#99] +(155) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#92] -(166) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#96] -Right keys [1]: [hd_demo_sk#99] -Join type: Inner -Join condition: None +(156) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90] +Right output [1]: [hd_demo_sk#92] +Arguments: [ss_hdemo_sk#89], [hd_demo_sk#92], Inner, BuildRight -(167) Project [codegen id : 38] -Output [2]: [ss_sold_time_sk#95, ss_store_sk#97] -Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, hd_demo_sk#99] +(157) CometProject +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, hd_demo_sk#92] +Arguments: [ss_sold_time_sk#88, ss_store_sk#90], [ss_sold_time_sk#88, ss_store_sk#90] -(168) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#100, t_hour#101, t_minute#102] +(158) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#93, t_hour#94, t_minute#95] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(169) CometFilter -Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] -Condition : ((((isnotnull(t_hour#101) AND isnotnull(t_minute#102)) AND (t_hour#101 = 12)) AND (t_minute#102 < 30)) AND isnotnull(t_time_sk#100)) - -(170) CometProject -Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] -Arguments: [t_time_sk#100], [t_time_sk#100] +(159) CometFilter +Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Condition : ((((isnotnull(t_hour#94) AND isnotnull(t_minute#95)) AND (t_hour#94 = 12)) AND (t_minute#95 < 30)) AND isnotnull(t_time_sk#93)) -(171) ColumnarToRow [codegen id : 36] -Input [1]: [t_time_sk#100] +(160) CometProject +Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Arguments: [t_time_sk#93], [t_time_sk#93] -(172) BroadcastExchange -Input [1]: [t_time_sk#100] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23] +(161) CometBroadcastExchange +Input [1]: [t_time_sk#93] +Arguments: [t_time_sk#93] -(173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#95] -Right keys [1]: [t_time_sk#100] -Join type: Inner -Join condition: None +(162) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#88, ss_store_sk#90] +Right output [1]: [t_time_sk#93] +Arguments: [ss_sold_time_sk#88], [t_time_sk#93], Inner, BuildRight -(174) Project [codegen id : 38] -Output [1]: [ss_store_sk#97] -Input [3]: [ss_sold_time_sk#95, ss_store_sk#97, t_time_sk#100] +(163) CometProject +Input [3]: [ss_sold_time_sk#88, ss_store_sk#90, t_time_sk#93] +Arguments: [ss_store_sk#90], [ss_store_sk#90] -(175) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#103] +(164) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#96] -(176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#97] -Right keys [1]: [s_store_sk#103] -Join type: Inner -Join condition: None +(165) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#90] +Right output [1]: [s_store_sk#96] +Arguments: [ss_store_sk#90], [s_store_sk#96], Inner, BuildRight -(177) Project [codegen id : 38] -Output: [] -Input [2]: [ss_store_sk#97, s_store_sk#103] +(166) CometProject +Input [2]: [ss_store_sk#90, s_store_sk#96] -(178) HashAggregate [codegen id : 38] +(167) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#104] -Results [1]: [count#105] -(179) Exchange -Input [1]: [count#105] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=24] +(168) ColumnarToRow [codegen id : 14] +Input [1]: [count#97] + +(169) Exchange +Input [1]: [count#97] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(180) HashAggregate [codegen id : 39] -Input [1]: [count#105] +(170) HashAggregate [codegen id : 15] +Input [1]: [count#97] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#106] -Results [1]: [count(1)#106 AS h12_to_12_30#107] +Aggregate Attributes [1]: [count(1)#98] +Results [1]: [count(1)#98 AS h12_to_12_30#99] -(181) BroadcastExchange -Input [1]: [h12_to_12_30#107] -Arguments: IdentityBroadcastMode, [plan_id=25] +(171) BroadcastExchange +Input [1]: [h12_to_12_30#99] +Arguments: IdentityBroadcastMode, [plan_id=15] -(182) BroadcastNestedLoopJoin [codegen id : 40] +(172) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt index b497e0bab6..b846d25d83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (40) +WholeStageCodegen (16) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin @@ -9,257 +9,203 @@ WholeStageCodegen (40) HashAggregate [count] [count(1),h8_30_to_9,count] InputAdapter Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometBroadcastExchange #2 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #3 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange #4 + CometProject [s_store_sk] + CometFilter [s_store_name,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [count] [count(1),h9_to_9_30,count] InputAdapter Exchange #6 - WholeStageCodegen (8) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #7 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #8 - WholeStageCodegen (14) + WholeStageCodegen (5) HashAggregate [count] [count(1),h9_30_to_10,count] InputAdapter Exchange #9 - WholeStageCodegen (13) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #10 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #11 - WholeStageCodegen (19) + WholeStageCodegen (7) HashAggregate [count] [count(1),h10_to_10_30,count] InputAdapter Exchange #12 - WholeStageCodegen (18) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #13 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #14 - WholeStageCodegen (24) + WholeStageCodegen (9) HashAggregate [count] [count(1),h10_30_to_11,count] InputAdapter Exchange #15 - WholeStageCodegen (23) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #16 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #17 - WholeStageCodegen (29) + WholeStageCodegen (11) HashAggregate [count] [count(1),h11_to_11_30,count] InputAdapter Exchange #18 - WholeStageCodegen (28) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #19 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #20 - WholeStageCodegen (34) + WholeStageCodegen (13) HashAggregate [count] [count(1),h11_30_to_12,count] InputAdapter Exchange #21 - WholeStageCodegen (33) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #22 - WholeStageCodegen (31) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #22 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #23 - WholeStageCodegen (39) + WholeStageCodegen (15) HashAggregate [count] [count(1),h12_to_12_30,count] InputAdapter Exchange #24 - WholeStageCodegen (38) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (14) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #25 - WholeStageCodegen (36) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #25 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt index 3657266e23..07761b14b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt @@ -1,31 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (27) -+- * Project (26) - +- * Filter (25) - +- Window (24) - +- * Sort (23) - +- Exchange (22) - +- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.store_sales (4) - : +- ReusedExchange (10) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store (13) +TakeOrderedAndProject (28) ++- * Project (27) + +- * Filter (26) + +- Window (25) + +- * Sort (24) + +- Exchange (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * ColumnarToRow (19) + +- CometProject (18) + +- CometBroadcastHashJoin (17) + :- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (7) + : : +- CometBroadcastHashJoin (6) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- CometBroadcastExchange (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.store_sales (3) + : +- CometBroadcastExchange (11) + : +- CometProject (10) + : +- CometFilter (9) + : +- CometScan parquet spark_catalog.default.date_dim (8) + +- CometBroadcastExchange (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) (1) Scan parquet spark_catalog.default.item @@ -39,10 +40,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] Condition : (isnotnull(ss_item_sk#5) AND isnotnull(ss_store_sk#6)) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] +Right output [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +Arguments: [i_item_sk#1], [ss_item_sk#5], Inner, BuildRight -(9) Project [codegen id : 4] -Output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +(7) CometProject Input [8]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +Arguments: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8], [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] + +(8) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct + +(9) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((isnotnull(d_year#11) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) + +(10) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Arguments: [d_date_sk#10, d_moy#12], [d_date_sk#10, d_moy#12] -(10) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_moy#11] +(11) CometBroadcastExchange +Input [2]: [d_date_sk#10, d_moy#12] +Arguments: [d_date_sk#10, d_moy#12] -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +Right output [2]: [d_date_sk#10, d_moy#12] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight -(12) Project [codegen id : 4] -Output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#11] -Input [8]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8, d_date_sk#10, d_moy#11] +(13) CometProject +Input [8]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8, d_date_sk#10, d_moy#12] +Arguments: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#12], [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#12] -(13) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +(14) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] -Condition : isnotnull(s_store_sk#12) +(15) CometFilter +Input [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] +Condition : isnotnull(s_store_sk#13) -(15) ColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +(16) CometBroadcastExchange +Input [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] +Arguments: [s_store_sk#13, s_store_name#14, s_company_name#15] -(16) BroadcastExchange -Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#12] +Right output [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] +Arguments: [ss_store_sk#6], [s_store_sk#13], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None +(18) CometProject +Input [9]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#12, s_store_sk#13, s_store_name#14, s_company_name#15] +Arguments: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15], [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15] -(18) Project [codegen id : 4] -Output [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#11, s_store_name#13, s_company_name#14] -Input [9]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15] -(19) HashAggregate [codegen id : 4] -Input [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#11, s_store_name#13, s_company_name#14] -Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11] +(20) HashAggregate [codegen id : 1] +Input [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15] +Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [1]: [sum#15] -Results [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum#17] -(20) Exchange -Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) Exchange +Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum#17] +Arguments: hashpartitioning(i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] -Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11] +(22) HashAggregate [codegen id : 2] +Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum#17] +Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12] Functions [1]: [sum(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#7))#17] -Results [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS _w0#19] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#7))#18] +Results [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#18,17,2) AS _w0#20] -(22) Exchange -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(23) Exchange +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#19, _w0#20] +Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#14, s_company_name#15, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(23) Sort [codegen id : 6] -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -Arguments: [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST], false, 0 +(24) Sort [codegen id : 3] +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#19, _w0#20] +Arguments: [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST, s_company_name#15 ASC NULLS FIRST], false, 0 -(24) Window -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#4, i_brand#2, s_store_name#13, s_company_name#14] +(25) Window +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#19, _w0#20] +Arguments: [avg(_w0#20) windowspecdefinition(i_category#4, i_brand#2, s_store_name#14, s_company_name#15, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#4, i_brand#2, s_store_name#14, s_company_name#15] -(25) Filter [codegen id : 7] -Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] -Condition : CASE WHEN NOT (avg_monthly_sales#20 = 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END +(26) Filter [codegen id : 4] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#19, _w0#20, avg_monthly_sales#21] +Condition : CASE WHEN NOT (avg_monthly_sales#21 = 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END -(26) Project [codegen id : 7] -Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] -Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] +(27) Project [codegen id : 4] +Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#19, avg_monthly_sales#21] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#19, _w0#20, avg_monthly_sales#21] -(27) TakeOrderedAndProject -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] +(28) TakeOrderedAndProject +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#19, avg_monthly_sales#21] +Arguments: 100, [(sum_sales#19 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#19, avg_monthly_sales#21] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (32) -+- * ColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan parquet spark_catalog.default.date_dim (28) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) -(28) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#21, d_moy#11] +(29) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(29) CometFilter -Input [3]: [d_date_sk#10, d_year#21, d_moy#11] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 1999)) AND isnotnull(d_date_sk#10)) +(30) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((isnotnull(d_year#11) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) -(30) CometProject -Input [3]: [d_date_sk#10, d_year#21, d_moy#11] -Arguments: [d_date_sk#10, d_moy#11], [d_date_sk#10, d_moy#11] +(31) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Arguments: [d_date_sk#10, d_moy#12], [d_date_sk#10, d_moy#12] -(31) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_moy#11] +(32) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#10, d_moy#12] -(32) BroadcastExchange -Input [2]: [d_date_sk#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) BroadcastExchange +Input [2]: [d_date_sk#10, d_moy#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt index bb9e4e17e2..52e066fe73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt @@ -1,50 +1,44 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,i_brand,s_store_name,s_company_name] InputAdapter Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometFilter [i_category,i_class,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt index fcfbca847a..dbea5e75de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt @@ -1,55 +1,51 @@ == Physical Plan == -* Project (51) -+- * BroadcastNestedLoopJoin Inner BuildRight (50) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (11) - : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : :- * ColumnarToRow (4) - : : : : +- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- BroadcastExchange (9) - : : : +- * ColumnarToRow (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.household_demographics (5) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan parquet spark_catalog.default.time_dim (12) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.web_page (19) - +- BroadcastExchange (49) - +- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * ColumnarToRow (32) - : : : +- CometProject (31) - : : : +- CometFilter (30) - : : : +- CometScan parquet spark_catalog.default.web_sales (29) - : : +- ReusedExchange (33) - : +- BroadcastExchange (40) - : +- * ColumnarToRow (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.time_dim (36) - +- ReusedExchange (43) +* Project (47) ++- * BroadcastNestedLoopJoin Inner BuildRight (46) + :- * HashAggregate (25) + : +- Exchange (24) + : +- * ColumnarToRow (23) + : +- CometHashAggregate (22) + : +- CometProject (21) + : +- CometBroadcastHashJoin (20) + : :- CometProject (15) + : : +- CometBroadcastHashJoin (14) + : : :- CometProject (9) + : : : +- CometBroadcastHashJoin (8) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.household_demographics (4) + : : +- CometBroadcastExchange (13) + : : +- CometProject (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.time_dim (10) + : +- CometBroadcastExchange (19) + : +- CometProject (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.web_page (16) + +- BroadcastExchange (45) + +- * HashAggregate (44) + +- Exchange (43) + +- * ColumnarToRow (42) + +- CometHashAggregate (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.web_sales (26) + : : +- ReusedExchange (29) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometFilter (33) + : +- CometScan parquet spark_catalog.default.time_dim (32) + +- ReusedExchange (38) (1) Scan parquet spark_catalog.default.web_sales @@ -67,226 +63,202 @@ Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AN Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -(4) ColumnarToRow [codegen id : 4] -Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(5) Scan parquet spark_catalog.default.household_demographics +(4) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#5, hd_dep_count#6] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [2]: [hd_demo_sk#5, hd_dep_count#6] Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) -(7) CometProject +(6) CometProject Input [2]: [hd_demo_sk#5, hd_dep_count#6] Arguments: [hd_demo_sk#5], [hd_demo_sk#5] -(8) ColumnarToRow [codegen id : 1] +(7) CometBroadcastExchange Input [1]: [hd_demo_sk#5] +Arguments: [hd_demo_sk#5] -(9) BroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(8) CometBroadcastHashJoin +Left output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] +Right output [1]: [hd_demo_sk#5] +Arguments: [ws_ship_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight -(10) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_ship_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 4] -Output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] +(9) CometProject Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] +Arguments: [ws_sold_time_sk#1, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_web_page_sk#3] -(12) Scan parquet spark_catalog.default.time_dim +(10) Scan parquet spark_catalog.default.time_dim Output [2]: [t_time_sk#7, t_hour#8] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] ReadSchema: struct -(13) CometFilter +(11) CometFilter Input [2]: [t_time_sk#7, t_hour#8] Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) -(14) CometProject +(12) CometProject Input [2]: [t_time_sk#7, t_hour#8] Arguments: [t_time_sk#7], [t_time_sk#7] -(15) ColumnarToRow [codegen id : 2] -Input [1]: [t_time_sk#7] - -(16) BroadcastExchange +(13) CometBroadcastExchange Input [1]: [t_time_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [t_time_sk#7] -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#7] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] +Right output [1]: [t_time_sk#7] +Arguments: [ws_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight -(18) Project [codegen id : 4] -Output [1]: [ws_web_page_sk#3] +(15) CometProject Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] +Arguments: [ws_web_page_sk#3], [ws_web_page_sk#3] -(19) Scan parquet spark_catalog.default.web_page +(16) Scan parquet spark_catalog.default.web_page Output [2]: [wp_web_page_sk#9, wp_char_count#10] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] ReadSchema: struct -(20) CometFilter +(17) CometFilter Input [2]: [wp_web_page_sk#9, wp_char_count#10] Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) -(21) CometProject +(18) CometProject Input [2]: [wp_web_page_sk#9, wp_char_count#10] Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] -(22) ColumnarToRow [codegen id : 3] +(19) CometBroadcastExchange Input [1]: [wp_web_page_sk#9] +Arguments: [wp_web_page_sk#9] -(23) BroadcastExchange -Input [1]: [wp_web_page_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_web_page_sk#3] -Right keys [1]: [wp_web_page_sk#9] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [1]: [ws_web_page_sk#3] +Right output [1]: [wp_web_page_sk#9] +Arguments: [ws_web_page_sk#3], [wp_web_page_sk#9], Inner, BuildRight -(25) Project [codegen id : 4] -Output: [] +(21) CometProject Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] -(26) HashAggregate [codegen id : 4] +(22) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#11] -Results [1]: [count#12] -(27) Exchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(23) ColumnarToRow [codegen id : 1] +Input [1]: [count#11] + +(24) Exchange +Input [1]: [count#11] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 10] -Input [1]: [count#12] +(25) HashAggregate [codegen id : 4] +Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#13] -Results [1]: [count(1)#13 AS amc#14] +Aggregate Attributes [1]: [count(1)#12] +Results [1]: [count(1)#12 AS amc#13] -(29) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] +(26) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] -Condition : ((isnotnull(ws_ship_hdemo_sk#16) AND isnotnull(ws_sold_time_sk#15)) AND isnotnull(ws_web_page_sk#17)) +(27) CometFilter +Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] +Condition : ((isnotnull(ws_ship_hdemo_sk#15) AND isnotnull(ws_sold_time_sk#14)) AND isnotnull(ws_web_page_sk#16)) -(31) CometProject -Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] -Arguments: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17], [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] +(28) CometProject +Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] +Arguments: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16], [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16] -(32) ColumnarToRow [codegen id : 8] -Input [3]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] +(29) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#18] -(33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#19] +(30) CometBroadcastHashJoin +Left output [3]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16] +Right output [1]: [hd_demo_sk#18] +Arguments: [ws_ship_hdemo_sk#15], [hd_demo_sk#18], Inner, BuildRight -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#16] -Right keys [1]: [hd_demo_sk#19] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#15, ws_web_page_sk#17] -Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, hd_demo_sk#19] +(31) CometProject +Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, hd_demo_sk#18] +Arguments: [ws_sold_time_sk#14, ws_web_page_sk#16], [ws_sold_time_sk#14, ws_web_page_sk#16] -(36) Scan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_hour#21] +(32) Scan parquet spark_catalog.default.time_dim +Output [2]: [t_time_sk#19, t_hour#20] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct -(37) CometFilter -Input [2]: [t_time_sk#20, t_hour#21] -Condition : (((isnotnull(t_hour#21) AND (t_hour#21 >= 19)) AND (t_hour#21 <= 20)) AND isnotnull(t_time_sk#20)) +(33) CometFilter +Input [2]: [t_time_sk#19, t_hour#20] +Condition : (((isnotnull(t_hour#20) AND (t_hour#20 >= 19)) AND (t_hour#20 <= 20)) AND isnotnull(t_time_sk#19)) -(38) CometProject -Input [2]: [t_time_sk#20, t_hour#21] -Arguments: [t_time_sk#20], [t_time_sk#20] +(34) CometProject +Input [2]: [t_time_sk#19, t_hour#20] +Arguments: [t_time_sk#19], [t_time_sk#19] -(39) ColumnarToRow [codegen id : 6] -Input [1]: [t_time_sk#20] +(35) CometBroadcastExchange +Input [1]: [t_time_sk#19] +Arguments: [t_time_sk#19] -(40) BroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) CometBroadcastHashJoin +Left output [2]: [ws_sold_time_sk#14, ws_web_page_sk#16] +Right output [1]: [t_time_sk#19] +Arguments: [ws_sold_time_sk#14], [t_time_sk#19], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#15] -Right keys [1]: [t_time_sk#20] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 8] -Output [1]: [ws_web_page_sk#17] -Input [3]: [ws_sold_time_sk#15, ws_web_page_sk#17, t_time_sk#20] +(37) CometProject +Input [3]: [ws_sold_time_sk#14, ws_web_page_sk#16, t_time_sk#19] +Arguments: [ws_web_page_sk#16], [ws_web_page_sk#16] -(43) ReusedExchange [Reuses operator id: 23] -Output [1]: [wp_web_page_sk#22] +(38) ReusedExchange [Reuses operator id: 19] +Output [1]: [wp_web_page_sk#21] -(44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#17] -Right keys [1]: [wp_web_page_sk#22] -Join type: Inner -Join condition: None +(39) CometBroadcastHashJoin +Left output [1]: [ws_web_page_sk#16] +Right output [1]: [wp_web_page_sk#21] +Arguments: [ws_web_page_sk#16], [wp_web_page_sk#21], Inner, BuildRight -(45) Project [codegen id : 8] -Output: [] -Input [2]: [ws_web_page_sk#17, wp_web_page_sk#22] +(40) CometProject +Input [2]: [ws_web_page_sk#16, wp_web_page_sk#21] -(46) HashAggregate [codegen id : 8] +(41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#23] -Results [1]: [count#24] -(47) Exchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(42) ColumnarToRow [codegen id : 2] +Input [1]: [count#22] + +(43) Exchange +Input [1]: [count#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(48) HashAggregate [codegen id : 9] -Input [1]: [count#24] +(44) HashAggregate [codegen id : 3] +Input [1]: [count#22] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#25] -Results [1]: [count(1)#25 AS pmc#26] +Aggregate Attributes [1]: [count(1)#23] +Results [1]: [count(1)#23 AS pmc#24] -(49) BroadcastExchange -Input [1]: [pmc#26] -Arguments: IdentityBroadcastMode, [plan_id=7] +(45) BroadcastExchange +Input [1]: [pmc#24] +Arguments: IdentityBroadcastMode, [plan_id=3] -(50) BroadcastNestedLoopJoin [codegen id : 10] +(46) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(51) Project [codegen id : 10] -Output [1]: [(cast(amc#14 as decimal(15,4)) / cast(pmc#26 as decimal(15,4))) AS am_pm_ratio#27] -Input [2]: [amc#14, pmc#26] +(47) Project [codegen id : 4] +Output [1]: [(cast(amc#13 as decimal(15,4)) / cast(pmc#24 as decimal(15,4))) AS am_pm_ratio#25] +Input [2]: [amc#13, pmc#24] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt index c4e04b06bc..50c8494fb5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt @@ -1,74 +1,56 @@ -WholeStageCodegen (10) +WholeStageCodegen (4) Project [amc,pmc] BroadcastNestedLoopJoin HashAggregate [count] [count(1),amc,count] InputAdapter Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] + CometBroadcastExchange #2 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange #3 + CometProject [t_time_sk] + CometFilter [t_hour,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + CometBroadcastExchange #4 + CometProject [wp_web_page_sk] + CometFilter [wp_char_count,wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [count] [count(1),pmc,count] InputAdapter Exchange #6 - WholeStageCodegen (8) - HashAggregate [count,count] - Project - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - InputAdapter - ReusedExchange [wp_web_page_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #7 + CometProject [t_time_sk] + CometFilter [t_hour,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt index 61f35489a2..4c0fcd87a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt @@ -1,47 +1,45 @@ == Physical Plan == -* Sort (43) -+- Exchange (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * Project (12) - : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.call_center (1) - : : : : : +- BroadcastExchange (7) - : : : : : +- * ColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (4) - : : : : +- ReusedExchange (10) - : : : +- BroadcastExchange (16) - : : : +- * ColumnarToRow (15) - : : : +- CometFilter (14) - : : : +- CometScan parquet spark_catalog.default.customer (13) - : : +- BroadcastExchange (23) - : : +- * ColumnarToRow (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan parquet spark_catalog.default.customer_address (19) - : +- BroadcastExchange (29) - : +- * ColumnarToRow (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.customer_demographics (26) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.household_demographics (32) +* Sort (41) ++- Exchange (40) + +- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * ColumnarToRow (36) + +- CometProject (35) + +- CometBroadcastHashJoin (34) + :- CometProject (29) + : +- CometBroadcastHashJoin (28) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (13) + : : : : +- CometBroadcastHashJoin (12) + : : : : :- CometProject (7) + : : : : : +- CometBroadcastHashJoin (6) + : : : : : :- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.call_center (1) + : : : : : +- CometBroadcastExchange (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (3) + : : : : +- CometBroadcastExchange (11) + : : : : +- CometProject (10) + : : : : +- CometFilter (9) + : : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : : +- CometBroadcastExchange (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.customer (14) + : : +- CometBroadcastExchange (22) + : : +- CometProject (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.customer_address (19) + : +- CometBroadcastExchange (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.customer_demographics (25) + +- CometBroadcastExchange (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.household_demographics (30) (1) Scan parquet spark_catalog.default.call_center @@ -55,10 +53,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] Condition : (isnotnull(cr_call_center_sk#6) AND isnotnull(cr_returning_customer_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] +Arguments: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] -(7) BroadcastExchange -Input [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] +Right output [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] +Arguments: [cc_call_center_sk#1], [cr_call_center_sk#6], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cc_call_center_sk#1] -Right keys [1]: [cr_call_center_sk#6] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 7] -Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8] +(7) CometProject Input [8]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8] + +(8) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] +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 + +(9) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 1998)) AND (d_moy#12 = 11)) AND isnotnull(d_date_sk#10)) -(10) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#10] +(10) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] -(11) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returned_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [cr_returned_date_sk#8], [d_date_sk#10], Inner, BuildRight -(12) Project [codegen id : 7] -Output [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7] +(13) CometProject Input [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8, d_date_sk#10] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7] -(13) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +(14) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] 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 -(14) CometFilter -Input [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] -Condition : (((isnotnull(c_customer_sk#11) AND isnotnull(c_current_addr_sk#14)) AND isnotnull(c_current_cdemo_sk#12)) AND isnotnull(c_current_hdemo_sk#13)) +(15) CometFilter +Input [4]: [c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] +Condition : (((isnotnull(c_customer_sk#13) AND isnotnull(c_current_addr_sk#16)) AND isnotnull(c_current_cdemo_sk#14)) AND isnotnull(c_current_hdemo_sk#15)) -(15) ColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +(16) CometBroadcastExchange +Input [4]: [c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] +Arguments: [c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] -(16) BroadcastExchange -Input [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7] +Right output [4]: [c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] +Arguments: [cr_returning_customer_sk#5], [c_customer_sk#13], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returning_customer_sk#5] -Right keys [1]: [c_customer_sk#11] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 7] -Output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] -Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +(18) CometProject +Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] (19) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#15, ca_gmt_offset#16] +Output [2]: [ca_address_sk#17, ca_gmt_offset#18] 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 (20) CometFilter -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] -Condition : ((isnotnull(ca_gmt_offset#16) AND (ca_gmt_offset#16 = -7.00)) AND isnotnull(ca_address_sk#15)) +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)) (21) CometProject -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] -Arguments: [ca_address_sk#15], [ca_address_sk#15] - -(22) ColumnarToRow [codegen id : 4] -Input [1]: [ca_address_sk#15] +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +Arguments: [ca_address_sk#17], [ca_address_sk#17] -(23) BroadcastExchange -Input [1]: [ca_address_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(22) CometBroadcastExchange +Input [1]: [ca_address_sk#17] +Arguments: [ca_address_sk#17] -(24) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#14] -Right keys [1]: [ca_address_sk#15] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] +Right output [1]: [ca_address_sk#17] +Arguments: [c_current_addr_sk#16], [ca_address_sk#17], Inner, BuildRight -(25) Project [codegen id : 7] -Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13] -Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14, ca_address_sk#15] +(24) CometProject +Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16, ca_address_sk#17] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15] -(26) Scan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +(25) Scan parquet spark_catalog.default.customer_demographics +Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] 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 -(27) CometFilter -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] -Condition : ((((cd_marital_status#18 = M) AND (cd_education_status#19 = Unknown )) OR ((cd_marital_status#18 = W) AND (cd_education_status#19 = Advanced Degree ))) AND isnotnull(cd_demo_sk#17)) - -(28) ColumnarToRow [codegen id : 5] -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +(26) CometFilter +Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Condition : ((((cd_marital_status#20 = M) AND (cd_education_status#21 = Unknown )) OR ((cd_marital_status#20 = W) AND (cd_education_status#21 = Advanced Degree ))) AND isnotnull(cd_demo_sk#19)) -(29) BroadcastExchange -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(27) CometBroadcastExchange +Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Arguments: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -(30) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#12] -Right keys [1]: [cd_demo_sk#17] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15] +Right output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Arguments: [c_current_cdemo_sk#14], [cd_demo_sk#19], Inner, BuildRight -(31) Project [codegen id : 7] -Output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#13, cd_marital_status#18, cd_education_status#19] -Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13, cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +(29) CometProject +Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15, cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#15, cd_marital_status#20, cd_education_status#21], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#15, cd_marital_status#20, cd_education_status#21] -(32) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] +(30) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#22, hd_buy_potential#23] 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 -(33) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((isnotnull(hd_buy_potential#21) AND StartsWith(hd_buy_potential#21, Unknown)) AND isnotnull(hd_demo_sk#20)) +(31) CometFilter +Input [2]: [hd_demo_sk#22, hd_buy_potential#23] +Condition : ((isnotnull(hd_buy_potential#23) AND StartsWith(hd_buy_potential#23, Unknown)) AND isnotnull(hd_demo_sk#22)) -(34) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] +(32) CometProject +Input [2]: [hd_demo_sk#22, hd_buy_potential#23] +Arguments: [hd_demo_sk#22], [hd_demo_sk#22] -(35) ColumnarToRow [codegen id : 6] -Input [1]: [hd_demo_sk#20] +(33) CometBroadcastExchange +Input [1]: [hd_demo_sk#22] +Arguments: [hd_demo_sk#22] -(36) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(34) CometBroadcastHashJoin +Left output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#15, cd_marital_status#20, cd_education_status#21] +Right output [1]: [hd_demo_sk#22] +Arguments: [c_current_hdemo_sk#15], [hd_demo_sk#22], Inner, BuildRight -(37) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#13] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None +(35) CometProject +Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#15, cd_marital_status#20, cd_education_status#21, hd_demo_sk#22] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21] -(38) Project [codegen id : 7] -Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#18, cd_education_status#19] -Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#13, cd_marital_status#18, cd_education_status#19, hd_demo_sk#20] +(36) ColumnarToRow [codegen id : 1] +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21] -(39) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#18, cd_education_status#19] -Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19] +(37) HashAggregate [codegen id : 1] +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21] +Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21] Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#7))] -Aggregate Attributes [1]: [sum#22] -Results [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] +Aggregate Attributes [1]: [sum#24] +Results [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21, sum#25] -(40) Exchange -Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] -Arguments: hashpartitioning(cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(38) Exchange +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21, sum#25] +Arguments: hashpartitioning(cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(41) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] -Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19] +(39) HashAggregate [codegen id : 2] +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21, sum#25] +Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21] Functions [1]: [sum(UnscaledValue(cr_net_loss#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#7))#24] -Results [4]: [cc_call_center_id#2 AS Call_Center#25, cc_name#3 AS Call_Center_Name#26, cc_manager#4 AS Manager#27, MakeDecimal(sum(UnscaledValue(cr_net_loss#7))#24,17,2) AS Returns_Loss#28] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#7))#26] +Results [4]: [cc_call_center_id#2 AS Call_Center#27, cc_name#3 AS Call_Center_Name#28, cc_manager#4 AS Manager#29, MakeDecimal(sum(UnscaledValue(cr_net_loss#7))#26,17,2) AS Returns_Loss#30] -(42) Exchange -Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] -Arguments: rangepartitioning(Returns_Loss#28 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(40) Exchange +Input [4]: [Call_Center#27, Call_Center_Name#28, Manager#29, Returns_Loss#30] +Arguments: rangepartitioning(Returns_Loss#30 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) Sort [codegen id : 9] -Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] -Arguments: [Returns_Loss#28 DESC NULLS LAST], true, 0 +(41) Sort [codegen id : 3] +Input [4]: [Call_Center#27, Call_Center_Name#28, Manager#29, Returns_Loss#30] +Arguments: [Returns_Loss#30 DESC NULLS LAST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +Subquery:1 Hosting operator id = 3 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) -(44) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#29, d_moy#30] +(42) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] 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 -(45) CometFilter -Input [3]: [d_date_sk#10, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 1998)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#10)) +(43) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 1998)) AND (d_moy#12 = 11)) AND isnotnull(d_date_sk#10)) -(46) CometProject -Input [3]: [d_date_sk#10, d_year#29, d_moy#30] +(44) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(47) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(48) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt index e5d62e3c0b..7e9b7b8bac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt @@ -1,73 +1,55 @@ -WholeStageCodegen (9) +WholeStageCodegen (3) Sort [Returns_Loss] InputAdapter Exchange [Returns_Loss] #1 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] InputAdapter Exchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - WholeStageCodegen (7) + WholeStageCodegen (1) 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 [c_current_hdemo_sk,hd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] + CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] CometFilter [cc_call_center_sk] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cr_call_center_sk,cr_returning_customer_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange #3 + CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometBroadcastExchange #7 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange #8 + CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastExchange #9 + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt index 5f1f961686..e8d482e9bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt @@ -1,33 +1,35 @@ == Physical Plan == -* HashAggregate (29) -+- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * HashAggregate (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * ColumnarToRow (13) - : : +- CometFilter (12) - : : +- CometScan parquet spark_catalog.default.web_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (24) +* HashAggregate (31) ++- Exchange (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * ColumnarToRow (9) + : : +- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.item (3) + : +- BroadcastExchange (23) + : +- * Filter (22) + : +- * HashAggregate (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- * ColumnarToRow (18) + : +- CometProject (17) + : +- CometBroadcastHashJoin (16) + : :- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.web_sales (10) + : +- CometBroadcastExchange (15) + : +- CometProject (14) + : +- CometFilter (13) + : +- CometScan parquet spark_catalog.default.date_dim (12) + +- ReusedExchange (26) (1) Scan parquet spark_catalog.default.web_sales @@ -42,42 +44,38 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) -(3) ColumnarToRow [codegen id : 6] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#5, i_manufact_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [i_item_sk#5, i_manufact_id#6] Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) -(6) CometProject +(5) CometProject Input [2]: [i_item_sk#5, i_manufact_id#6] Arguments: [i_item_sk#5], [i_item_sk#5] -(7) ColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [1]: [i_item_sk#5] +Arguments: [i_item_sk#5] -(8) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Right output [1]: [i_item_sk#5] +Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] +(8) CometProject Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] +Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5], [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] + +(9) ColumnarToRow [codegen id : 4] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -(11) Scan parquet spark_catalog.default.web_sales +(10) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -85,125 +83,140 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(12) CometFilter +(11) CometFilter Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] Condition : isnotnull(ws_item_sk#7) -(13) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] +(12) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#11] +(13) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] -(16) Project [codegen id : 3] -Output [2]: [ws_item_sk#7, ws_ext_discount_amt#8] +(15) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(16) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] +Right output [1]: [d_date_sk#11] +Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight + +(17) CometProject Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#11] +Arguments: [ws_item_sk#7, ws_ext_discount_amt#8], [ws_item_sk#7, ws_ext_discount_amt#8] + +(18) ColumnarToRow [codegen id : 1] +Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -(17) HashAggregate [codegen id : 3] +(19) HashAggregate [codegen id : 1] Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] Keys [1]: [ws_item_sk#7] Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] -Aggregate Attributes [2]: [sum#12, count#13] -Results [3]: [ws_item_sk#7, sum#14, count#15] +Aggregate Attributes [2]: [sum#13, count#14] +Results [3]: [ws_item_sk#7, sum#15, count#16] -(18) Exchange -Input [3]: [ws_item_sk#7, sum#14, count#15] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(20) Exchange +Input [3]: [ws_item_sk#7, sum#15, count#16] +Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(19) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#7, sum#14, count#15] +(21) HashAggregate [codegen id : 2] +Input [3]: [ws_item_sk#7, sum#15, count#16] Keys [1]: [ws_item_sk#7] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#16] -Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#17] +Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#17 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#7] -(20) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) +(22) Filter [codegen id : 2] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#7] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#18) -(21) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] +(23) BroadcastExchange +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=2] -(22) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#5] Right keys [1]: [ws_item_sk#7] Join type: Inner -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#18) -(23) Project [codegen id : 6] +(25) Project [codegen id : 4] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#18, ws_item_sk#7] -(24) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#18] +(26) ReusedExchange [Reuses operator id: 36] +Output [1]: [d_date_sk#19] -(25) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#18] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#19] -(27) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 4] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#19] -Results [1]: [sum#20] +Aggregate Attributes [1]: [sum#20] +Results [1]: [sum#21] -(28) Exchange -Input [1]: [sum#20] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(30) Exchange +Input [1]: [sum#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(29) HashAggregate [codegen id : 7] -Input [1]: [sum#20] +(31) HashAggregate [codegen id : 5] +Input [1]: [sum#21] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#21] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#21,17,2) AS Excess Discount Amount #22] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#22] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#22,17,2) AS Excess Discount Amount #23] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (34) -+- * ColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.date_dim (30) +BroadcastExchange (36) ++- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(30) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#23] +(32) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#19, d_date#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [d_date_sk#18, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) +(33) CometFilter +Input [2]: [d_date_sk#19, d_date#24] +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 2000-01-27)) AND (d_date#24 <= 2000-04-26)) AND isnotnull(d_date_sk#19)) -(32) CometProject -Input [2]: [d_date_sk#18, d_date#23] -Arguments: [d_date_sk#18], [d_date_sk#18] +(34) CometProject +Input [2]: [d_date_sk#19, d_date#24] +Arguments: [d_date_sk#19], [d_date_sk#19] -(33) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] +(35) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#19] -(34) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) BroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 10 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt index a5e724c1ff..bcc1ef1b5f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt @@ -1,17 +1,17 @@ -WholeStageCodegen (7) +WholeStageCodegen (5) HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] InputAdapter Exchange #1 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ws_ext_discount_amt] [sum,sum] Project [ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_ext_discount_amt,ws_sold_date_sk] BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] - Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk,ws_ext_discount_amt] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -22,31 +22,29 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange #3 + CometProject [i_item_sk] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Filter [(1.3 * avg(ws_ext_discount_amt))] HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] InputAdapter Exchange [ws_item_sk] #5 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] - Project [ws_item_sk,ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt index 60c262e9c4..4bc24750f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt @@ -1,32 +1,29 @@ == Physical Plan == -* HashAggregate (28) -+- Exchange (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.household_demographics (5) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometProject (14) - : +- CometFilter (13) - : +- CometScan parquet spark_catalog.default.time_dim (12) - +- BroadcastExchange (23) - +- * ColumnarToRow (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan parquet spark_catalog.default.store (19) +* HashAggregate (25) ++- Exchange (24) + +- * ColumnarToRow (23) + +- CometHashAggregate (22) + +- CometProject (21) + +- CometBroadcastHashJoin (20) + :- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.household_demographics (4) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.time_dim (10) + +- CometBroadcastExchange (19) + +- CometProject (18) + +- CometFilter (17) + +- CometScan parquet spark_catalog.default.store (16) (1) Scan parquet spark_catalog.default.store_sales @@ -44,120 +41,105 @@ Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isn Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -(4) ColumnarToRow [codegen id : 4] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(5) Scan parquet spark_catalog.default.household_demographics +(4) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#5, hd_dep_count#6] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [2]: [hd_demo_sk#5, hd_dep_count#6] Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) -(7) CometProject +(6) CometProject Input [2]: [hd_demo_sk#5, hd_dep_count#6] Arguments: [hd_demo_sk#5], [hd_demo_sk#5] -(8) ColumnarToRow [codegen id : 1] -Input [1]: [hd_demo_sk#5] - -(9) BroadcastExchange +(7) CometBroadcastExchange Input [1]: [hd_demo_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [hd_demo_sk#5] -(10) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Right output [1]: [hd_demo_sk#5] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight -(11) Project [codegen id : 4] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +(9) CometProject Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] -(12) Scan parquet spark_catalog.default.time_dim +(10) Scan parquet spark_catalog.default.time_dim Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(13) CometFilter +(11) CometFilter Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) -(14) CometProject +(12) CometProject Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] Arguments: [t_time_sk#7], [t_time_sk#7] -(15) ColumnarToRow [codegen id : 2] -Input [1]: [t_time_sk#7] - -(16) BroadcastExchange +(13) CometBroadcastExchange Input [1]: [t_time_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [t_time_sk#7] -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#7] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Right output [1]: [t_time_sk#7] +Arguments: [ss_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight -(18) Project [codegen id : 4] -Output [1]: [ss_store_sk#3] +(15) CometProject Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] +Arguments: [ss_store_sk#3], [ss_store_sk#3] -(19) Scan parquet spark_catalog.default.store +(16) Scan parquet spark_catalog.default.store Output [2]: [s_store_sk#10, s_store_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] ReadSchema: struct -(20) CometFilter +(17) CometFilter Input [2]: [s_store_sk#10, s_store_name#11] Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) -(21) CometProject +(18) CometProject Input [2]: [s_store_sk#10, s_store_name#11] Arguments: [s_store_sk#10], [s_store_sk#10] -(22) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#10] - -(23) BroadcastExchange +(19) CometBroadcastExchange Input [1]: [s_store_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [s_store_sk#10] -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#3] +Right output [1]: [s_store_sk#10] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(25) Project [codegen id : 4] -Output: [] +(21) CometProject Input [2]: [ss_store_sk#3, s_store_sk#10] -(26) HashAggregate [codegen id : 4] +(22) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#12] -Results [1]: [count#13] -(27) Exchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(23) ColumnarToRow [codegen id : 1] +Input [1]: [count#12] + +(24) Exchange +Input [1]: [count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 5] -Input [1]: [count#13] +(25) HashAggregate [codegen id : 2] +Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#14] -Results [1]: [count(1)#14 AS count(1)#15] +Aggregate Attributes [1]: [count(1)#13] +Results [1]: [count(1)#13 AS count(1)#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt index d1438f48eb..6149152260 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt @@ -1,41 +1,29 @@ -WholeStageCodegen (5) +WholeStageCodegen (2) HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometBroadcastExchange #2 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange #3 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange #4 + CometProject [s_store_sk] + CometFilter [s_store_name,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt index 66ccf4f223..7508405a85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt @@ -1,27 +1,30 @@ == Physical Plan == -* HashAggregate (23) -+- Exchange (22) - +- * HashAggregate (21) - +- * Project (20) - +- * SortMergeJoin FullOuter (19) - :- * Sort (9) - : +- * HashAggregate (8) - : +- Exchange (7) - : +- * HashAggregate (6) - : +- * Project (5) - : +- * BroadcastHashJoin Inner BuildRight (4) - : :- * ColumnarToRow (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (3) - +- * Sort (18) - +- * HashAggregate (17) - +- Exchange (16) - +- * HashAggregate (15) - +- * Project (14) - +- * BroadcastHashJoin Inner BuildRight (13) - :- * ColumnarToRow (11) - : +- CometScan parquet spark_catalog.default.catalog_sales (10) - +- ReusedExchange (12) +* HashAggregate (26) ++- Exchange (25) + +- * HashAggregate (24) + +- * Project (23) + +- * SortMergeJoin FullOuter (22) + :- * Sort (12) + : +- * HashAggregate (11) + : +- Exchange (10) + : +- * ColumnarToRow (9) + : +- CometHashAggregate (8) + : +- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (5) + : +- CometProject (4) + : +- CometFilter (3) + : +- CometScan parquet spark_catalog.default.date_dim (2) + +- * Sort (21) + +- * HashAggregate (20) + +- Exchange (19) + +- * ColumnarToRow (18) + +- CometHashAggregate (17) + +- CometProject (16) + +- CometBroadcastHashJoin (15) + :- CometScan parquet spark_catalog.default.catalog_sales (13) + +- ReusedExchange (14) (1) Scan parquet spark_catalog.default.store_sales @@ -31,149 +34,159 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] +(2) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(3) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#5] +(3) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(4) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(4) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(5) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] -(5) Project [codegen id : 2] -Output [2]: [ss_item_sk#1, ss_customer_sk#2] +(6) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(7) CometProject Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] +Arguments: [ss_item_sk#1, ss_customer_sk#2], [ss_item_sk#1, ss_customer_sk#2] -(6) HashAggregate [codegen id : 2] +(8) CometHashAggregate Input [2]: [ss_item_sk#1, ss_customer_sk#2] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] -Aggregate Attributes: [] -Results [2]: [ss_customer_sk#2, ss_item_sk#1] -(7) Exchange +(9) ColumnarToRow [codegen id : 1] +Input [2]: [ss_customer_sk#2, ss_item_sk#1] + +(10) Exchange Input [2]: [ss_customer_sk#2, ss_item_sk#1] Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(8) HashAggregate [codegen id : 3] +(11) HashAggregate [codegen id : 2] Input [2]: [ss_customer_sk#2, ss_item_sk#1] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] Aggregate Attributes: [] -Results [2]: [ss_customer_sk#2 AS customer_sk#6, ss_item_sk#1 AS item_sk#7] +Results [2]: [ss_customer_sk#2 AS customer_sk#7, ss_item_sk#1 AS item_sk#8] -(9) Sort [codegen id : 3] -Input [2]: [customer_sk#6, item_sk#7] -Arguments: [customer_sk#6 ASC NULLS FIRST, item_sk#7 ASC NULLS FIRST], false, 0 +(12) Sort [codegen id : 2] +Input [2]: [customer_sk#7, item_sk#8] +Arguments: [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST], false, 0 -(10) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] +(13) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 5] -Input [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] +(14) ReusedExchange [Reuses operator id: 5] +Output [1]: [d_date_sk#13] -(12) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#12] +(15) CometBroadcastHashJoin +Left output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(13) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 5] -Output [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Input [4]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10, d_date_sk#12] +(16) CometProject +Input [4]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11, d_date_sk#13] +Arguments: [cs_bill_customer_sk#9, cs_item_sk#10], [cs_bill_customer_sk#9, cs_item_sk#10] -(15) HashAggregate [codegen id : 5] -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] +(17) CometHashAggregate +Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] +Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Functions: [] -Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -(16) Exchange -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Arguments: hashpartitioning(cs_bill_customer_sk#8, cs_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(18) ColumnarToRow [codegen id : 3] +Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] + +(19) Exchange +Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] +Arguments: hashpartitioning(cs_bill_customer_sk#9, cs_item_sk#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) HashAggregate [codegen id : 6] -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] +(20) HashAggregate [codegen id : 4] +Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] +Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#8 AS customer_sk#13, cs_item_sk#9 AS item_sk#14] +Results [2]: [cs_bill_customer_sk#9 AS customer_sk#14, cs_item_sk#10 AS item_sk#15] -(18) Sort [codegen id : 6] -Input [2]: [customer_sk#13, item_sk#14] -Arguments: [customer_sk#13 ASC NULLS FIRST, item_sk#14 ASC NULLS FIRST], false, 0 +(21) Sort [codegen id : 4] +Input [2]: [customer_sk#14, item_sk#15] +Arguments: [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 7] -Left keys [2]: [customer_sk#6, item_sk#7] -Right keys [2]: [customer_sk#13, item_sk#14] +(22) SortMergeJoin [codegen id : 5] +Left keys [2]: [customer_sk#7, item_sk#8] +Right keys [2]: [customer_sk#14, item_sk#15] Join type: FullOuter Join condition: None -(20) Project [codegen id : 7] -Output [2]: [customer_sk#6, customer_sk#13] -Input [4]: [customer_sk#6, item_sk#7, customer_sk#13, item_sk#14] +(23) Project [codegen id : 5] +Output [2]: [customer_sk#7, customer_sk#14] +Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] -(21) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#6, customer_sk#13] +(24) HashAggregate [codegen id : 5] +Input [2]: [customer_sk#7, customer_sk#14] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#15, sum#16, sum#17] -Results [3]: [sum#18, sum#19, sum#20] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#16, sum#17, sum#18] +Results [3]: [sum#19, sum#20, sum#21] -(22) Exchange -Input [3]: [sum#18, sum#19, sum#20] +(25) Exchange +Input [3]: [sum#19, sum#20, sum#21] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(23) HashAggregate [codegen id : 8] -Input [3]: [sum#18, sum#19, sum#20] +(26) HashAggregate [codegen id : 6] +Input [3]: [sum#19, sum#20, sum#21] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END)#21, sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#22, sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#23] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END)#21 AS store_only#24, sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#22 AS catalog_only#25, sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#23 AS store_and_catalog#26] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END)#22, sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#23, sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#24] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END)#22 AS store_only#25, sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#23 AS catalog_only#26, sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#24 AS store_and_catalog#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * ColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.date_dim (24) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(24) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#27] +(27) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#27] -Condition : (((isnotnull(d_month_seq#27) AND (d_month_seq#27 >= 1200)) AND (d_month_seq#27 <= 1211)) AND isnotnull(d_date_sk#5)) +(28) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(26) CometProject -Input [2]: [d_date_sk#5, d_month_seq#27] +(29) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(27) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(28) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt index be9c20a560..24e6dceef3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (8) +WholeStageCodegen (6) HashAggregate [sum,sum,sum] [sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),store_only,catalog_only,store_and_catalog,sum,sum,sum] InputAdapter Exchange #1 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [customer_sk,customer_sk] [sum,sum,sum,sum,sum,sum] Project [customer_sk,customer_sk] SortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] InputAdapter - WholeStageCodegen (3) + WholeStageCodegen (2) Sort [customer_sk,item_sk] HashAggregate [ss_customer_sk,ss_item_sk] [customer_sk,item_sk] InputAdapter Exchange [ss_customer_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_customer_sk,ss_item_sk] + CometProject [ss_item_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,21 +27,22 @@ WholeStageCodegen (8) CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (4) Sort [customer_sk,item_sk] HashAggregate [cs_bill_customer_sk,cs_item_sk] [customer_sk,item_sk] InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #4 - WholeStageCodegen (5) - HashAggregate [cs_bill_customer_sk,cs_item_sk] - Project [cs_bill_customer_sk,cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [cs_bill_customer_sk,cs_item_sk] + CometProject [cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt index 3d66a07d07..af9d0026a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt @@ -1,26 +1,28 @@ == Physical Plan == -* Project (22) -+- * Sort (21) - +- Exchange (20) - +- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +* Project (24) ++- * Sort (23) + +- Exchange (22) + +- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * HashAggregate (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.store_sales @@ -35,126 +37,137 @@ ReadSchema: struct Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 27] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [ss_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] +Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(15) HashAggregate [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Aggregate Attributes [1]: [sum#13] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] -(14) Exchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +(17) HashAggregate [codegen id : 2] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] -Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#15] +Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#15,17,2) AS itemrevenue#16, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#15,17,2) AS _w0#17, i_item_id#6] -(16) Exchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(18) Exchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +(19) Sort [codegen id : 3] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(20) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6] +Arguments: [sum(_w0#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9] -(19) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] -Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] +(21) Project [codegen id : 4] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, ((_w0#17 * 100) / _we0#18) AS revenueratio#19, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, i_item_id#6, _we0#18] -(20) Exchange -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(22) Exchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19, i_item_id#6] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#19 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) Sort [codegen id : 7] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], true, 0 +(23) Sort [codegen id : 5] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19, i_item_id#6] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#19 ASC NULLS FIRST], true, 0 -(22) Project [codegen id : 7] -Output [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +(24) Project [codegen id : 5] +Output [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19, i_item_id#6] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) -(23) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(26) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(25) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(27) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt index 9eabb9977c..84d2778863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt @@ -1,28 +1,28 @@ -WholeStageCodegen (7) +WholeStageCodegen (5) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] InputAdapter Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #2 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -33,12 +33,10 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #5 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt index 9365550266..a6282e50c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt @@ -1,36 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.ship_mode (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.call_center (16) - +- BroadcastExchange (26) - +- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +TakeOrderedAndProject (28) ++- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometProject (17) + : +- CometBroadcastHashJoin (16) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.warehouse (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.ship_mode (8) + : +- CometBroadcastExchange (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.call_center (13) + +- CometBroadcastExchange (21) + +- CometProject (20) + +- CometFilter (19) + +- CometScan parquet spark_catalog.default.date_dim (18) (1) Scan parquet spark_catalog.default.catalog_sales @@ -44,144 +40,128 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Condition : isnotnull(w_warehouse_sk#6) -(6) ColumnarToRow [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [w_warehouse_sk#6, w_warehouse_name#7] -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_warehouse_sk#4] -Right keys [1]: [w_warehouse_sk#6] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] +Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [cs_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight -(9) Project [codegen id : 5] -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] +(7) CometProject Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] -(10) Scan parquet spark_catalog.default.ship_mode +(8) Scan parquet spark_catalog.default.ship_mode Output [2]: [sm_ship_mode_sk#8, sm_type#9] Batched: true Location [not included in comparison]/{warehouse_dir}/ship_mode] PushedFilters: [IsNotNull(sm_ship_mode_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [sm_ship_mode_sk#8, sm_type#9] Condition : isnotnull(sm_ship_mode_sk#8) -(12) ColumnarToRow [codegen id : 2] -Input [2]: [sm_ship_mode_sk#8, sm_type#9] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [sm_ship_mode_sk#8, sm_type#9] -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_ship_mode_sk#3] -Right keys [1]: [sm_ship_mode_sk#8] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] +Right output [2]: [sm_ship_mode_sk#8, sm_type#9] +Arguments: [cs_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight -(15) Project [codegen id : 5] -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9] +(12) CometProject Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#9] +Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9] -(16) Scan parquet spark_catalog.default.call_center +(13) Scan parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#10, cc_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_call_center_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [2]: [cc_call_center_sk#10, cc_name#11] Condition : isnotnull(cc_call_center_sk#10) -(18) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [2]: [cc_call_center_sk#10, cc_name#11] +Arguments: [cc_call_center_sk#10, cc_name#11] -(19) BroadcastExchange -Input [2]: [cc_call_center_sk#10, cc_name#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_call_center_sk#2] -Right keys [1]: [cc_call_center_sk#10] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9] +Right output [2]: [cc_call_center_sk#10, cc_name#11] +Arguments: [cs_call_center_sk#2], [cc_call_center_sk#10], Inner, BuildRight -(21) Project [codegen id : 5] -Output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11] +(17) CometProject Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_call_center_sk#10, cc_name#11] +Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11], [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11] -(22) Scan parquet spark_catalog.default.date_dim +(18) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_month_seq#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(23) CometFilter +(19) CometFilter Input [2]: [d_date_sk#12, d_month_seq#13] Condition : (((isnotnull(d_month_seq#13) AND (d_month_seq#13 >= 1200)) AND (d_month_seq#13 <= 1211)) AND isnotnull(d_date_sk#12)) -(24) CometProject +(20) CometProject Input [2]: [d_date_sk#12, d_month_seq#13] Arguments: [d_date_sk#12], [d_date_sk#12] -(25) ColumnarToRow [codegen id : 4] -Input [1]: [d_date_sk#12] - -(26) BroadcastExchange +(21) CometBroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: [d_date_sk#12] -(27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11] +Right output [1]: [d_date_sk#12] +Arguments: [cs_ship_date_sk#1], [d_date_sk#12], Inner, BuildRight -(28) Project [codegen id : 5] -Output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] +(23) CometProject Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11, d_date_sk#12] +Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupingexpression#14], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] + +(24) ColumnarToRow [codegen id : 1] +Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupingexpression#14] -(29) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 1] Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupingexpression#14] Keys [3]: [_groupingexpression#14, sm_type#9, cc_name#11] Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] Results [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -(30) Exchange +(26) Exchange Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(31) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 2] Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] Keys [3]: [_groupingexpression#14, sm_type#9, cc_name#11] Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] Aggregate Attributes [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] -(32) TakeOrderedAndProject +(28) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt index c5f25f0795..da4ef22e01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt @@ -1,48 +1,32 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter Exchange [_groupingexpression,sm_type,cc_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,w_warehouse_name] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] + CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] + CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #2 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #3 + CometFilter [sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange #4 + CometFilter [cc_call_center_sk] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt index 59efef88e6..1e60e40c5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt @@ -1,44 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Filter (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_returns (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (24) - : : +- * Filter (23) - : : +- * HashAggregate (22) - : : +- Exchange (21) - : : +- * HashAggregate (20) - : : +- * HashAggregate (19) - : : +- Exchange (18) - : : +- * HashAggregate (17) - : : +- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * ColumnarToRow (13) - : : : +- CometFilter (12) - : : : +- CometScan parquet spark_catalog.default.store_returns (11) - : : +- ReusedExchange (14) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.store (27) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer (34) +TakeOrderedAndProject (43) ++- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Filter (13) + : : : +- * HashAggregate (12) + : : : +- Exchange (11) + : : : +- * ColumnarToRow (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_returns (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (27) + : : +- * Filter (26) + : : +- * HashAggregate (25) + : : +- Exchange (24) + : : +- * HashAggregate (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * ColumnarToRow (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.store_returns (14) + : : +- ReusedExchange (16) + : +- BroadcastExchange (34) + : +- * ColumnarToRow (33) + : +- CometProject (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.store (30) + +- BroadcastExchange (40) + +- * ColumnarToRow (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.customer (37) (1) Scan parquet spark_catalog.default.store_returns @@ -53,45 +56,58 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_year#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(6) Project [codegen id : 2] -Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] +(7) CometBroadcastHashJoin +Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] +Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -(7) HashAggregate [codegen id : 2] +(9) CometHashAggregate Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -(8) Exchange +(10) ColumnarToRow [codegen id : 1] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] + +(11) Exchange Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) HashAggregate [codegen id : 9] +(12) HashAggregate [codegen id : 7] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(10) Filter [codegen id : 9] +(13) Filter [codegen id : 7] Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] Condition : isnotnull(ctr_total_return#12) -(11) Scan parquet spark_catalog.default.store_returns +(14) Scan parquet spark_catalog.default.store_returns Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] @@ -99,176 +115,173 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(12) CometFilter +(15) CometFilter Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Condition : isnotnull(sr_store_sk#2) -(13) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] - -(14) ReusedExchange [Reuses operator id: 45] +(16) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#6] -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight -(16) Project [codegen id : 4] -Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] +(18) CometProject Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] +Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -(17) HashAggregate [codegen id : 4] +(19) CometHashAggregate Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum#14] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] -(18) Exchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] +(20) ColumnarToRow [codegen id : 2] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] + +(21) Exchange +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(19) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#15] +(22) HashAggregate [codegen id : 3] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] Results [2]: [sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(20) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [2]: [ctr_store_sk#11, ctr_total_return#12] Keys [1]: [ctr_store_sk#11] Functions [1]: [partial_avg(ctr_total_return#12)] -Aggregate Attributes [2]: [sum#16, count#17] -Results [3]: [ctr_store_sk#11, sum#18, count#19] +Aggregate Attributes [2]: [sum#15, count#16] +Results [3]: [ctr_store_sk#11, sum#17, count#18] -(21) Exchange -Input [3]: [ctr_store_sk#11, sum#18, count#19] +(24) Exchange +Input [3]: [ctr_store_sk#11, sum#17, count#18] Arguments: hashpartitioning(ctr_store_sk#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(22) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#11, sum#18, count#19] +(25) HashAggregate [codegen id : 4] +Input [3]: [ctr_store_sk#11, sum#17, count#18] Keys [1]: [ctr_store_sk#11] Functions [1]: [avg(ctr_total_return#12)] -Aggregate Attributes [1]: [avg(ctr_total_return#12)#20] -Results [2]: [(avg(ctr_total_return#12)#20 * 1.2) AS (avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11 AS ctr_store_sk#11#22] +Aggregate Attributes [1]: [avg(ctr_total_return#12)#19] +Results [2]: [(avg(ctr_total_return#12)#19 * 1.2) AS (avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11 AS ctr_store_sk#11#21] -(23) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#21) +(26) Filter [codegen id : 4] +Input [2]: [(avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11#21] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#20) -(24) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] +(27) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11#21] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] -(25) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [ctr_store_sk#11#22] +Right keys [1]: [ctr_store_sk#11#21] Join type: Inner -Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#21) +Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#20) -(26) Project [codegen id : 9] +(29) Project [codegen id : 7] Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] -Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#21, ctr_store_sk#11#22] +Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11#21] -(27) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_state#24] +(30) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#22, s_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [s_store_sk#23, s_state#24] -Condition : ((isnotnull(s_state#24) AND (s_state#24 = TN)) AND isnotnull(s_store_sk#23)) +(31) CometFilter +Input [2]: [s_store_sk#22, s_state#23] +Condition : ((isnotnull(s_state#23) AND (s_state#23 = TN)) AND isnotnull(s_store_sk#22)) -(29) CometProject -Input [2]: [s_store_sk#23, s_state#24] -Arguments: [s_store_sk#23], [s_store_sk#23] +(32) CometProject +Input [2]: [s_store_sk#22, s_state#23] +Arguments: [s_store_sk#22], [s_store_sk#22] -(30) ColumnarToRow [codegen id : 7] -Input [1]: [s_store_sk#23] +(33) ColumnarToRow [codegen id : 5] +Input [1]: [s_store_sk#22] -(31) BroadcastExchange -Input [1]: [s_store_sk#23] +(34) BroadcastExchange +Input [1]: [s_store_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(32) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [s_store_sk#23] +Right keys [1]: [s_store_sk#22] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(36) Project [codegen id : 7] Output [1]: [ctr_customer_sk#10] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#23] +Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#22] -(34) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#25, c_customer_id#26] +(37) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#24, c_customer_id#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [c_customer_sk#25, c_customer_id#26] -Condition : isnotnull(c_customer_sk#25) +(38) CometFilter +Input [2]: [c_customer_sk#24, c_customer_id#25] +Condition : isnotnull(c_customer_sk#24) -(36) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#25, c_customer_id#26] +(39) ColumnarToRow [codegen id : 6] +Input [2]: [c_customer_sk#24, c_customer_id#25] -(37) BroadcastExchange -Input [2]: [c_customer_sk#25, c_customer_id#26] +(40) BroadcastExchange +Input [2]: [c_customer_sk#24, c_customer_id#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(38) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_customer_sk#10] -Right keys [1]: [c_customer_sk#25] +Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(39) Project [codegen id : 9] -Output [1]: [c_customer_id#26] -Input [3]: [ctr_customer_sk#10, c_customer_sk#25, c_customer_id#26] +(42) Project [codegen id : 7] +Output [1]: [c_customer_id#25] +Input [3]: [ctr_customer_sk#10, c_customer_sk#24, c_customer_id#25] -(40) TakeOrderedAndProject -Input [1]: [c_customer_id#26] -Arguments: 100, [c_customer_id#26 ASC NULLS FIRST], [c_customer_id#26] +(43) TakeOrderedAndProject +Input [1]: [c_customer_id#25] +Arguments: 100, [c_customer_id#25 ASC NULLS FIRST], [c_customer_id#25] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(41) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#27] +(44) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_year#7] 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 -(42) CometFilter -Input [2]: [d_date_sk#6, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2000)) AND isnotnull(d_date_sk#6)) +(45) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(43) CometProject -Input [2]: [d_date_sk#6, d_year#27] +(46) CometProject +Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(44) ColumnarToRow [codegen id : 1] +(47) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(45) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 14 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt index 6d4c0fca7a..000160518a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (9) + WholeStageCodegen (7) Project [c_customer_id] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk] @@ -10,12 +10,12 @@ TakeOrderedAndProject [c_customer_id] HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] InputAdapter Exchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometFilter [sr_store_sk,sr_customer_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -26,42 +26,43 @@ TakeOrderedAndProject [c_customer_id] CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #3 - WholeStageCodegen (6) + BroadcastExchange #4 + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_store_sk,sum,count] InputAdapter - Exchange [ctr_store_sk] #4 - WholeStageCodegen (5) + Exchange [ctr_store_sk] #5 + WholeStageCodegen (3) HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] InputAdapter - Exchange [sr_customer_sk,sr_store_sk] #5 - WholeStageCodegen (4) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [sr_customer_sk,sr_store_sk] #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [s_store_sk] CometFilter [s_state,s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #8 + WholeStageCodegen (6) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 15490b87d7..4a29b72602 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -1,47 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (37) - +- * ColumnarToRow (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.customer_demographics (34) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * Project (28) + : : +- * Filter (27) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) + : : : :- * ColumnarToRow (12) + : : : : +- CometBroadcastHashJoin (11) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (10) + : : : : +- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (33) + : +- * ColumnarToRow (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (39) + +- * ColumnarToRow (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.customer_demographics (36) (1) Scan parquet spark_catalog.default.customer @@ -55,232 +57,241 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#9] +(5) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(6) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(7) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] + +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] +(9) CometProject Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ss_customer_sk#6], [ss_customer_sk#6] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#6] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Right output [1]: [ss_customer_sk#6] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight + +(12) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(13) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#15] -(13) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#13] +(15) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +Right output [1]: [d_date_sk#15] +Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(16) CometProject +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] +Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +(17) ColumnarToRow [codegen id : 1] +Input [1]: [ws_bill_customer_sk#12] -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) BroadcastExchange +Input [1]: [ws_bill_customer_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 9] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] +Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#19] -(20) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#17] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(23) CometProject +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#14] -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] +(24) ColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#16] -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#14] +Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(27) Filter [codegen id : 5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(28) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(27) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_county#19] +(29) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [ca_address_sk#18, ca_county#19] -Condition : (ca_county#19 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#18)) +(30) CometFilter +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) -(29) CometProject -Input [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] +(31) CometProject +Input [2]: [ca_address_sk#20, ca_county#21] +Arguments: [ca_address_sk#20], [ca_address_sk#20] -(30) ColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#18] +(32) ColumnarToRow [codegen id : 3] +Input [1]: [ca_address_sk#20] -(31) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(33) BroadcastExchange +Input [1]: [ca_address_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(32) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(35) Project [codegen id : 5] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] -(34) Scan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(36) Scan parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(35) CometFilter -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#20) +(37) CometFilter +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#22) -(36) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(38) ColumnarToRow [codegen id : 4] +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(37) BroadcastExchange -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(39) BroadcastExchange +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(38) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#20] +Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(39) Project [codegen id : 9] -Output [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(41) Project [codegen id : 5] +Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(40) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(42) HashAggregate [codegen id : 5] +Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] +Aggregate Attributes [1]: [count#31] +Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32] -(41) Exchange -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(43) Exchange +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32] +Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(42) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(44) HashAggregate [codegen id : 6] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#32] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] +Aggregate Attributes [1]: [count(1)#33] +Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#33 AS cnt1#34, cd_purchase_estimate#26, count(1)#33 AS cnt2#35, cd_credit_rating#27, count(1)#33 AS cnt3#36, cd_dep_count#28, count(1)#33 AS cnt4#37, cd_dep_employed_count#29, count(1)#33 AS cnt5#38, cd_dep_college_count#30, count(1)#33 AS cnt6#39] -(43) TakeOrderedAndProject -Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] -Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] +(45) TakeOrderedAndProject +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#34, cd_purchase_estimate#26, cnt2#35, cd_credit_rating#27, cnt3#36, cd_dep_count#28, cnt4#37, cd_dep_employed_count#29, cnt5#38, cd_dep_college_count#30, cnt6#39] +Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#34, cd_purchase_estimate#26, cnt2#35, cd_credit_rating#27, cnt3#36, cd_dep_count#28, cnt4#37, cd_dep_employed_count#29, cnt5#38, cd_dep_college_count#30, cnt6#39] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(44) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#38, d_moy#39] +(46) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [3]: [d_date_sk#9, d_year#38, d_moy#39] -Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 1)) AND (d_moy#39 <= 4)) AND isnotnull(d_date_sk#9)) +(47) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) -(46) CometProject -Input [3]: [d_date_sk#9, d_year#38, d_moy#39] +(48) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(47) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(48) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index 89893c831e..19243e359d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) + WholeStageCodegen (6) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,62 +13,58 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Filter [exists,exists] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [ca_address_sk] CometFilter [ca_county,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index c663d46886..1c5d5222e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt @@ -2,75 +2,75 @@ TakeOrderedAndProject (72) +- * Project (71) +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) - : :- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (51) - : +- * Filter (50) - : +- * HashAggregate (49) - : +- Exchange (48) - : +- * HashAggregate (47) - : +- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (43) - : : +- * BroadcastHashJoin Inner BuildRight (42) - : : :- * ColumnarToRow (37) - : : : +- CometFilter (36) - : : : +- CometScan parquet spark_catalog.default.customer (35) - : : +- BroadcastExchange (41) - : : +- * ColumnarToRow (40) - : : +- CometFilter (39) - : : +- CometScan parquet spark_catalog.default.web_sales (38) - : +- ReusedExchange (44) + :- * Project (54) + : +- * BroadcastHashJoin Inner BuildRight (53) + : :- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (17) + : : : +- * HashAggregate (16) + : : : +- Exchange (15) + : : : +- * ColumnarToRow (14) + : : : +- CometHashAggregate (13) + : : : +- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- BroadcastExchange (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * ColumnarToRow (31) + : : +- CometHashAggregate (30) + : : +- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- CometBroadcastExchange (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- BroadcastExchange (52) + : +- * Filter (51) + : +- * HashAggregate (50) + : +- Exchange (49) + : +- * ColumnarToRow (48) + : +- CometHashAggregate (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (43) + : : +- CometBroadcastHashJoin (42) + : : :- CometFilter (38) + : : : +- CometScan parquet spark_catalog.default.customer (37) + : : +- CometBroadcastExchange (41) + : : +- CometFilter (40) + : : +- CometScan parquet spark_catalog.default.web_sales (39) + : +- ReusedExchange (44) +- BroadcastExchange (69) +- * HashAggregate (68) +- Exchange (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * ColumnarToRow (56) - : : +- CometFilter (55) - : : +- CometScan parquet spark_catalog.default.customer (54) - : +- BroadcastExchange (60) - : +- * ColumnarToRow (59) - : +- CometFilter (58) - : +- CometScan parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (63) + +- * ColumnarToRow (66) + +- CometHashAggregate (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometFilter (56) + : : +- CometScan parquet spark_catalog.default.customer (55) + : +- CometBroadcastExchange (59) + : +- CometFilter (58) + : +- CometScan parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (62) (1) Scan parquet spark_catalog.default.customer @@ -84,10 +84,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] Condition : isnotnull(ss_customer_sk#9) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(7) BroadcastExchange -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Right output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: [c_customer_sk#1], [ss_customer_sk#9], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +(7) CometProject Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(10) ReusedExchange [Reuses operator id: 76] +(8) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_year#15] +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 -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) + +(10) CometBroadcastExchange +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14, d_year#15] -(12) Project [codegen id : 3] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] +(11) CometBroadcastHashJoin +Left output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Right output [2]: [d_date_sk#14, d_year#15] +Arguments: [ss_sold_date_sk#12], [d_date_sk#14], Inner, BuildRight + +(12) CometProject Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#14, d_year#15] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] -(13) HashAggregate [codegen id : 3] +(13) CometHashAggregate Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] -Aggregate Attributes [1]: [sum#16] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -(14) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(14) ColumnarToRow [codegen id : 1] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#16] + +(15) Exchange +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#16] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 16] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] +(16) HashAggregate [codegen id : 8] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#16] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] -Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#17] +Results [2]: [c_customer_id#2 AS customer_id#18, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#17,18,2) AS year_total#19] -(16) Filter [codegen id : 16] -Input [2]: [customer_id#19, year_total#20] -Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) +(17) Filter [codegen id : 8] +Input [2]: [customer_id#18, year_total#19] +Condition : (isnotnull(year_total#19) AND (year_total#19 > 0.00)) -(17) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +(18) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter -Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) - -(19) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +(19) CometFilter +Input [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_customer_id#21)) (20) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#33)] +PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (21) CometFilter -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -Condition : isnotnull(ss_customer_sk#29) +Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Condition : isnotnull(ss_customer_sk#28) -(22) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +(22) CometBroadcastExchange +Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Arguments: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -(23) BroadcastExchange -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Right output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Arguments: [c_customer_sk#20], [ss_customer_sk#28], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#21] -Right keys [1]: [ss_customer_sk#29] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] - -(26) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#34, d_year#35] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(24) CometProject +Input [12]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31], [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] -Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] - -(29) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] -Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] -Aggregate Attributes [1]: [sum#36] -Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] - -(30) Exchange -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] -Results [3]: [c_customer_id#22 AS customer_id#38, c_preferred_cust_flag#25 AS customer_preferred_cust_flag#39, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#40] - -(32) BroadcastExchange -Input [3]: [customer_id#38, customer_preferred_cust_flag#39, year_total#40] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#33, d_year#34] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct -(33) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#38] +(26) CometFilter +Input [2]: [d_date_sk#33, d_year#34] +Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) + +(27) CometBroadcastExchange +Input [2]: [d_date_sk#33, d_year#34] +Arguments: [d_date_sk#33, d_year#34] + +(28) CometBroadcastHashJoin +Left output [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Right output [2]: [d_date_sk#33, d_year#34] +Arguments: [ss_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight + +(29) CometProject +Input [12]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31, d_date_sk#33, d_year#34] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34], [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] + +(30) CometHashAggregate +Input [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] +Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] + +(31) ColumnarToRow [codegen id : 2] +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#35] + +(32) Exchange +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#35] +Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(33) HashAggregate [codegen id : 3] +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#35] +Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))#17] +Results [3]: [c_customer_id#21 AS customer_id#36, c_preferred_cust_flag#24 AS customer_preferred_cust_flag#37, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))#17,18,2) AS year_total#38] + +(34) BroadcastExchange +Input [3]: [customer_id#36, customer_preferred_cust_flag#37, year_total#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(35) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#36] Join type: Inner Join condition: None -(34) Project [codegen id : 16] -Output [4]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40] -Input [5]: [customer_id#19, year_total#20, customer_id#38, customer_preferred_cust_flag#39, year_total#40] +(36) Project [codegen id : 8] +Output [4]: [customer_id#18, year_total#19, customer_preferred_cust_flag#37, year_total#38] +Input [5]: [customer_id#18, year_total#19, customer_id#36, customer_preferred_cust_flag#37, year_total#38] -(35) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +(37) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#39, c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(36) CometFilter -Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] -Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_customer_id#42)) - -(37) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +(38) CometFilter +Input [8]: [c_customer_sk#39, c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46] +Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_customer_id#40)) -(38) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +(39) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#53)] +PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#51)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(39) CometFilter -Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -Condition : isnotnull(ws_bill_customer_sk#49) +(40) CometFilter +Input [4]: [ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50] +Condition : isnotnull(ws_bill_customer_sk#47) -(40) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +(41) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50] +Arguments: [ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50] -(41) BroadcastExchange -Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(42) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#39, c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46] +Right output [4]: [ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50] +Arguments: [c_customer_sk#39], [ws_bill_customer_sk#47], Inner, BuildRight -(42) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#41] -Right keys [1]: [ws_bill_customer_sk#49] -Join type: Inner -Join condition: None +(43) CometProject +Input [12]: [c_customer_sk#39, c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_bill_customer_sk#47, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50] +Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50], [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50] -(43) Project [codegen id : 10] -Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -Input [12]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +(44) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#52, d_year#53] -(44) ReusedExchange [Reuses operator id: 76] -Output [2]: [d_date_sk#54, d_year#55] +(45) CometBroadcastHashJoin +Left output [10]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50] +Right output [2]: [d_date_sk#52, d_year#53] +Arguments: [ws_sold_date_sk#50], [d_date_sk#52], Inner, BuildRight -(45) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#54] -Join type: Inner -Join condition: None +(46) CometProject +Input [12]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, ws_sold_date_sk#50, d_date_sk#52, d_year#53] +Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, d_year#53], [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, d_year#53] + +(47) CometHashAggregate +Input [10]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, ws_ext_discount_amt#48, ws_ext_list_price#49, d_year#53] +Keys [8]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))] + +(48) ColumnarToRow [codegen id : 4] +Input [9]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, sum#54] -(46) Project [codegen id : 10] -Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] -Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52, d_date_sk#54, d_year#55] - -(47) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] -Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] -Aggregate Attributes [1]: [sum#56] -Results [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] - -(48) Exchange -Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] -Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(49) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] -Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58] -Results [2]: [c_customer_id#42 AS customer_id#59, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58,18,2) AS year_total#60] - -(50) Filter [codegen id : 11] -Input [2]: [customer_id#59, year_total#60] -Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.00)) - -(51) BroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(52) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#59] +(49) Exchange +Input [9]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, sum#54] +Arguments: hashpartitioning(c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(50) HashAggregate [codegen id : 5] +Input [9]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53, sum#54] +Keys [8]: [c_customer_id#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_country#44, c_login#45, c_email_address#46, d_year#53] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))#55] +Results [2]: [c_customer_id#40 AS customer_id#56, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#49 - ws_ext_discount_amt#48)))#55,18,2) AS year_total#57] + +(51) Filter [codegen id : 5] +Input [2]: [customer_id#56, year_total#57] +Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) + +(52) BroadcastExchange +Input [2]: [customer_id#56, year_total#57] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(53) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#56] Join type: Inner Join condition: None -(53) Project [codegen id : 16] -Output [5]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60] -Input [6]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, customer_id#59, year_total#60] +(54) Project [codegen id : 8] +Output [5]: [customer_id#18, year_total#19, customer_preferred_cust_flag#37, year_total#38, year_total#57] +Input [6]: [customer_id#18, year_total#19, customer_preferred_cust_flag#37, year_total#38, customer_id#56, year_total#57] -(54) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] +(55) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(55) CometFilter -Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] -Condition : (isnotnull(c_customer_sk#61) AND isnotnull(c_customer_id#62)) - -(56) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] +(56) CometFilter +Input [8]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65] +Condition : (isnotnull(c_customer_sk#58) AND isnotnull(c_customer_id#59)) (57) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (58) CometFilter -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_bill_customer_sk#69) +Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Condition : isnotnull(ws_bill_customer_sk#66) -(59) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +(59) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Arguments: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -(60) BroadcastExchange -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(60) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65] +Right output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Arguments: [c_customer_sk#58], [ws_bill_customer_sk#66], Inner, BuildRight -(61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#61] -Right keys [1]: [ws_bill_customer_sk#69] -Join type: Inner -Join condition: None +(61) CometProject +Input [12]: [c_customer_sk#58, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Arguments: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69], [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -(62) Project [codegen id : 14] -Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -Input [12]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +(62) ReusedExchange [Reuses operator id: 27] +Output [2]: [d_date_sk#71, d_year#72] -(63) ReusedExchange [Reuses operator id: 80] -Output [2]: [d_date_sk#74, d_year#75] +(63) CometBroadcastHashJoin +Left output [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Right output [2]: [d_date_sk#71, d_year#72] +Arguments: [ws_sold_date_sk#69], [d_date_sk#71], Inner, BuildRight -(64) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#74] -Join type: Inner -Join condition: None +(64) CometProject +Input [12]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69, d_date_sk#71, d_year#72] +Arguments: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72], [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] -(65) Project [codegen id : 14] -Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] -Input [12]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72, d_date_sk#74, d_year#75] +(65) CometHashAggregate +Input [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] +Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] -(66) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] -Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] -Aggregate Attributes [1]: [sum#76] -Results [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] +(66) ColumnarToRow [codegen id : 6] +Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, sum#73] (67) Exchange -Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] -Arguments: hashpartitioning(c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, sum#73] +Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(68) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] -Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#58] -Results [2]: [c_customer_id#62 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#58,18,2) AS year_total#79] +(68) HashAggregate [codegen id : 7] +Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72, sum#73] +Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#63, c_login#64, c_email_address#65, d_year#72] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))#55] +Results [2]: [c_customer_id#59 AS customer_id#74, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))#55,18,2) AS year_total#75] (69) BroadcastExchange -Input [2]: [customer_id#78, year_total#79] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +Input [2]: [customer_id#74, year_total#75] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(70) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#78] +(70) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#74] Join type: Inner -Join condition: (CASE WHEN (year_total#60 > 0.00) THEN (year_total#79 / year_total#60) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END) +Join condition: (CASE WHEN (year_total#57 > 0.00) THEN (year_total#75 / year_total#57) END > CASE WHEN (year_total#19 > 0.00) THEN (year_total#38 / year_total#19) END) -(71) Project [codegen id : 16] -Output [1]: [customer_preferred_cust_flag#39] -Input [7]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60, customer_id#78, year_total#79] +(71) Project [codegen id : 8] +Output [1]: [customer_preferred_cust_flag#37] +Input [7]: [customer_id#18, year_total#19, customer_preferred_cust_flag#37, year_total#38, year_total#57, customer_id#74, year_total#75] (72) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#39] -Arguments: 100, [customer_preferred_cust_flag#39 ASC NULLS FIRST], [customer_preferred_cust_flag#39] +Input [1]: [customer_preferred_cust_flag#37] +Arguments: 100, [customer_preferred_cust_flag#37 ASC NULLS FIRST], [customer_preferred_cust_flag#37] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (76) +- * ColumnarToRow (75) +- CometFilter (74) @@ -448,9 +444,9 @@ Input [2]: [d_date_sk#14, d_year#15] (76) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 BroadcastExchange (80) +- * ColumnarToRow (79) +- CometFilter (78) @@ -458,25 +454,25 @@ BroadcastExchange (80) (77) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#34, d_year#35] +Output [2]: [d_date_sk#33, d_year#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (78) CometFilter -Input [2]: [d_date_sk#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) +Input [2]: [d_date_sk#33, d_year#34] +Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) (79) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_year#35] +Input [2]: [d_date_sk#33, d_year#34] (80) BroadcastExchange -Input [2]: [d_date_sk#34, d_year#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Input [2]: [d_date_sk#33, d_year#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#33 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#32 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt index 562b5fdf29..dd1a522069 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_preferred_cust_flag] - WholeStageCodegen (16) + WholeStageCodegen (8) Project [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] @@ -10,114 +10,98 @@ TakeOrderedAndProject [customer_preferred_cust_flag] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange #7 CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #13 + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 6cf7f4b088..160f0c593b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * ColumnarToRow (15) + +- CometHashAggregate (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.web_sales @@ -33,118 +35,127 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [ws_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) CometHashAggregate Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(15) ColumnarToRow [codegen id : 1] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] -(16) Exchange +(18) Exchange Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] +(19) Sort [codegen id : 3] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window +(20) Window Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(19) Project [codegen id : 6] +(21) Project [codegen id : 4] Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] -(20) TakeOrderedAndProject +(22) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) -(21) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(23) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index fae1c6dba1..905a35c8b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,12 +29,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt index a647b1f053..1cda122617 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt @@ -1,38 +1,37 @@ == Physical Plan == -* HashAggregate (34) -+- Exchange (33) - +- * HashAggregate (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store (4) - : : : +- BroadcastExchange (14) - : : : +- * ColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (17) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.customer_demographics (20) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.household_demographics (26) +* HashAggregate (33) ++- Exchange (32) + +- * ColumnarToRow (31) + +- CometHashAggregate (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometProject (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.customer_address (8) + : : +- CometBroadcastExchange (17) + : : +- CometProject (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.date_dim (14) + : +- CometBroadcastExchange (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.customer_demographics (20) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.household_demographics (25) (1) Scan parquet spark_catalog.default.store_sales @@ -47,186 +46,183 @@ ReadSchema: struct= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) -(3) ColumnarToRow [codegen id : 6] -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] - -(4) Scan parquet spark_catalog.default.store +(3) Scan parquet spark_catalog.default.store Output [1]: [s_store_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [1]: [s_store_sk#12] Condition : isnotnull(s_store_sk#12) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [1]: [s_store_sk#12] +Arguments: [s_store_sk#12] -(7) BroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +Right output [1]: [s_store_sk#12] +Arguments: [ss_store_sk#4], [s_store_sk#12], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 6] -Output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +(7) CometProject Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] +Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -(10) Scan parquet spark_catalog.default.customer_address +(8) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] 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, [OH,TX]),In(ca_state, [KY,NM,OR])),In(ca_state, [MS,TX,VA]))] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((ca_state#14 IN (TX,OH) OR ca_state#14 IN (OR,NM,KY)) OR ca_state#14 IN (VA,TX,MS))) -(12) CometProject +(10) CometProject Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] Arguments: [ca_address_sk#13, ca_state#14], [ca_address_sk#13, ca_state#14] -(13) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#13, ca_state#14] - -(14) BroadcastExchange +(11) CometBroadcastExchange Input [2]: [ca_address_sk#13, ca_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [ca_address_sk#13, ca_state#14] -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#13] -Join type: Inner -Join condition: ((((ca_state#14 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#14 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#14 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) +(12) CometBroadcastHashJoin +Left output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +Right output [2]: [ca_address_sk#13, ca_state#14] +Arguments: [ss_addr_sk#3], [ca_address_sk#13], Inner, ((((ca_state#14 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#14 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#14 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))), BuildRight -(16) Project [codegen id : 6] -Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] +(13) CometProject Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#14] +Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] + +(14) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_year#17] +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 + +(15) CometFilter +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) + +(16) CometProject +Input [2]: [d_date_sk#16, d_year#17] +Arguments: [d_date_sk#16], [d_date_sk#16] -(17) ReusedExchange [Reuses operator id: 39] -Output [1]: [d_date_sk#16] +(17) CometBroadcastExchange +Input [1]: [d_date_sk#16] +Arguments: [d_date_sk#16] -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] +Right output [1]: [d_date_sk#16] +Arguments: [ss_sold_date_sk#10], [d_date_sk#16], Inner, BuildRight -(19) Project [codegen id : 6] -Output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] +(19) CometProject Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#16] +Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] (20) Scan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (21) CometFilter -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] -Condition : (isnotnull(cd_demo_sk#17) AND ((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) OR ((cd_marital_status#18 = S) AND (cd_education_status#19 = College ))) OR ((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )))) - -(22) ColumnarToRow [codegen id : 4] -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] +Condition : (isnotnull(cd_demo_sk#18) AND ((((cd_marital_status#19 = M) AND (cd_education_status#20 = Advanced Degree )) OR ((cd_marital_status#19 = S) AND (cd_education_status#20 = College ))) OR ((cd_marital_status#19 = W) AND (cd_education_status#20 = 2 yr Degree )))) -(23) BroadcastExchange -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(22) CometBroadcastExchange +Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] +Arguments: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#17] -Join type: Inner -Join condition: ((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +(23) CometBroadcastHashJoin +Left output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] +Right output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] +Arguments: [ss_cdemo_sk#1], [cd_demo_sk#18], Inner, ((((((cd_marital_status#19 = M) AND (cd_education_status#20 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#19 = S) AND (cd_education_status#20 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#19 = W) AND (cd_education_status#20 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))), BuildRight -(25) Project [codegen id : 6] -Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19] -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +(24) CometProject +Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] +Arguments: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20], [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20] -(26) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_dep_count#21] +(25) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#21, hd_dep_count#22] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] ReadSchema: struct -(27) CometFilter -Input [2]: [hd_demo_sk#20, hd_dep_count#21] -Condition : (isnotnull(hd_demo_sk#20) AND ((hd_dep_count#21 = 3) OR (hd_dep_count#21 = 1))) - -(28) ColumnarToRow [codegen id : 5] -Input [2]: [hd_demo_sk#20, hd_dep_count#21] +(26) CometFilter +Input [2]: [hd_demo_sk#21, hd_dep_count#22] +Condition : (isnotnull(hd_demo_sk#21) AND ((hd_dep_count#22 = 3) OR (hd_dep_count#22 = 1))) -(29) BroadcastExchange -Input [2]: [hd_demo_sk#20, hd_dep_count#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(27) CometBroadcastExchange +Input [2]: [hd_demo_sk#21, hd_dep_count#22] +Arguments: [hd_demo_sk#21, hd_dep_count#22] -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: (((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#21 = 3)) OR (((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#21 = 1))) OR (((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#21 = 1))) +(28) CometBroadcastHashJoin +Left output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20] +Right output [2]: [hd_demo_sk#21, hd_dep_count#22] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#21], Inner, (((((((cd_marital_status#19 = M) AND (cd_education_status#20 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#22 = 3)) OR (((((cd_marital_status#19 = S) AND (cd_education_status#20 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#22 = 1))) OR (((((cd_marital_status#19 = W) AND (cd_education_status#20 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#22 = 1))), BuildRight -(31) Project [codegen id : 6] -Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19, hd_demo_sk#20, hd_dep_count#21] +(29) CometProject +Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20, hd_demo_sk#21, hd_dep_count#22] +Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -(32) HashAggregate [codegen id : 6] +(30) CometHashAggregate Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Keys: [] Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [7]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28] -Results [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] -(33) Exchange -Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] +(31) ColumnarToRow [codegen id : 1] +Input [7]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29] -(34) HashAggregate [codegen id : 7] -Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] +(32) Exchange +Input [7]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] + +(33) HashAggregate [codegen id : 2] +Input [7]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [4]: [avg(ss_quantity#5)#36, avg(UnscaledValue(ss_ext_sales_price#7))#37, avg(UnscaledValue(ss_ext_wholesale_cost#8))#38, sum(UnscaledValue(ss_ext_wholesale_cost#8))#39] -Results [4]: [avg(ss_quantity#5)#36 AS avg(ss_quantity)#40, cast((avg(UnscaledValue(ss_ext_sales_price#7))#37 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#41, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#38 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#42, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#39,17,2) AS sum(ss_ext_wholesale_cost)#43] +Aggregate Attributes [4]: [avg(ss_quantity#5)#30, avg(UnscaledValue(ss_ext_sales_price#7))#31, avg(UnscaledValue(ss_ext_wholesale_cost#8))#32, sum(UnscaledValue(ss_ext_wholesale_cost#8))#33] +Results [4]: [avg(ss_quantity#5)#30 AS avg(ss_quantity)#34, cast((avg(UnscaledValue(ss_ext_sales_price#7))#31 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#35, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#32 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#36, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#33,17,2) AS sum(ss_ext_wholesale_cost)#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (39) -+- * ColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan parquet spark_catalog.default.date_dim (35) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(35) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_year#44] +(34) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_year#17] 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 -(36) CometFilter -Input [2]: [d_date_sk#16, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2001)) AND isnotnull(d_date_sk#16)) +(35) CometFilter +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(37) CometProject -Input [2]: [d_date_sk#16, d_year#44] +(36) CometProject +Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(38) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] -(39) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt index 5e5fc41f83..4de403664d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt @@ -1,21 +1,21 @@ -WholeStageCodegen (7) +WholeStageCodegen (2) HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] InputAdapter Exchange #1 - 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_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] - Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + CometProject [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometProject [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_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,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -26,34 +26,20 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_dep_count] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #4 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #6 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastExchange #7 + CometFilter [hd_demo_sk,hd_dep_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 627a77a885..0b4712fc08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -28,43 +28,43 @@ TakeOrderedAndProject (105) : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) : : : :- * HashAggregate (35) : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) + : : : : +- * ColumnarToRow (33) + : : : : +- CometHashAggregate (32) + : : : : +- CometProject (31) + : : : : +- CometBroadcastHashJoin (30) + : : : : :- CometProject (28) + : : : : : +- CometBroadcastHashJoin (27) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- CometBroadcastExchange (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (10) + : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometProject (23) + : : : : : +- CometBroadcastHashJoin (22) + : : : : : :- CometProject (17) + : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : :- CometFilter (12) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : +- CometBroadcastExchange (15) + : : : : : : +- CometFilter (14) + : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : +- CometBroadcastExchange (21) + : : : : : +- CometProject (20) + : : : : : +- CometFilter (19) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) + : : : : +- ReusedExchange (29) : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * ColumnarToRow (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) + : : : +- * ColumnarToRow (44) + : : : +- CometProject (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (38) + : : : +- ReusedExchange (41) : : +- BroadcastExchange (57) : : +- * BroadcastHashJoin LeftSemi BuildRight (56) : : :- * ColumnarToRow (54) @@ -118,7 +118,7 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 25] +(3) ColumnarToRow [codegen id : 11] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (4) Scan parquet spark_catalog.default.item @@ -132,7 +132,7 @@ ReadSchema: struct Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -172,629 +166,623 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(19) BroadcastExchange +(15) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(17) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#24] +(18) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(19) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(20) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] + +(21) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] + +(22) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight + +(23) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(24) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(25) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(28) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#25] +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(31) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] (34) Exchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] +Results [3]: [brand_id#27, class_id#28, category_id#29] (36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (37) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(42) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#36] +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] (45) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(48) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(49) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] (50) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [ss_item_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (52) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (53) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : isnotnull(i_item_sk#38) +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : isnotnull(i_item_sk#39) -(54) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] (55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (57) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(58) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(59) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 129] -Output [1]: [d_date_sk#42] +(60) ReusedExchange [Reuses operator id: 127] +Output [1]: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(62) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(63) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] (64) Exchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(65) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(65) HashAggregate [codegen id : 12] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#51, count(1)#50 AS number_sales#52] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#52, count(1)#51 AS number_sales#53] -(66) Filter [codegen id : 26] -Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) +(66) Filter [codegen id : 12] +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(67) Project [codegen id : 26] -Output [6]: [sales#51, number_sales#52, store AS channel#55, i_brand_id#39 AS i_brand_id#56, i_class_id#40 AS i_class_id#57, i_category_id#41 AS i_category_id#58] -Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] +(67) Project [codegen id : 12] +Output [6]: [sales#52, number_sales#53, store AS channel#56, i_brand_id#40 AS i_brand_id#57, i_class_id#41 AS i_class_id#58, i_category_id#42 AS i_category_id#59] +Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#52, number_sales#53] (68) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +Output [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#62), dynamicpruningexpression(cs_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(cs_sold_date_sk#63), dynamicpruningexpression(cs_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (69) CometFilter -Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] -Condition : isnotnull(cs_item_sk#59) +Input [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] +Condition : isnotnull(cs_item_sk#60) -(70) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +(70) ColumnarToRow [codegen id : 23] +Input [4]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63] (71) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(72) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#59] -Right keys [1]: [ss_item_sk#37] +(72) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#60] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (73) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +Output [4]: [i_item_sk#65, i_brand_id#66, i_class_id#67, i_category_id#68] -(74) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#59] -Right keys [1]: [i_item_sk#64] +(74) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#60] +Right keys [1]: [i_item_sk#65] Join type: Inner Join condition: None -(75) Project [codegen id : 51] -Output [6]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [8]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +(75) Project [codegen id : 23] +Output [6]: [cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_brand_id#66, i_class_id#67, i_category_id#68] +Input [8]: [cs_item_sk#60, cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_item_sk#65, i_brand_id#66, i_class_id#67, i_category_id#68] -(76) ReusedExchange [Reuses operator id: 129] -Output [1]: [d_date_sk#68] +(76) ReusedExchange [Reuses operator id: 127] +Output [1]: [d_date_sk#69] -(77) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#62] -Right keys [1]: [d_date_sk#68] +(77) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_sold_date_sk#63] +Right keys [1]: [d_date_sk#69] Join type: Inner Join condition: None -(78) Project [codegen id : 51] -Output [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [7]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] +(78) Project [codegen id : 23] +Output [5]: [cs_quantity#61, cs_list_price#62, i_brand_id#66, i_class_id#67, i_category_id#68] +Input [7]: [cs_quantity#61, cs_list_price#62, cs_sold_date_sk#63, i_brand_id#66, i_class_id#67, i_category_id#68, d_date_sk#69] -(79) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#65, i_class_id#66, i_category_id#67] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [partial_sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), partial_count(1)] -Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +(79) HashAggregate [codegen id : 23] +Input [5]: [cs_quantity#61, cs_list_price#62, i_brand_id#66, i_class_id#67, i_category_id#68] +Keys [3]: [i_brand_id#66, i_class_id#67, i_category_id#68] +Functions [2]: [partial_sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62)), partial_count(1)] +Aggregate Attributes [3]: [sum#70, isEmpty#71, count#72] +Results [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] (80) Exchange -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] +Arguments: hashpartitioning(i_brand_id#66, i_class_id#67, i_category_id#68, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(81) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#75, count(1)#76] -Results [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#75 AS sales#77, count(1)#76 AS number_sales#78] +(81) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] +Keys [3]: [i_brand_id#66, i_class_id#67, i_category_id#68] +Functions [2]: [sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62))#76, count(1)#77] +Results [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum((cast(cs_quantity#61 as decimal(10,0)) * cs_list_price#62))#76 AS sales#78, count(1)#77 AS number_sales#79] -(82) Filter [codegen id : 52] -Input [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#77, number_sales#78] -Condition : (isnotnull(sales#77) AND (cast(sales#77 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) +(82) Filter [codegen id : 24] +Input [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sales#78, number_sales#79] +Condition : (isnotnull(sales#78) AND (cast(sales#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(83) Project [codegen id : 52] -Output [6]: [sales#77, number_sales#78, catalog AS channel#79, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [5]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#77, number_sales#78] +(83) Project [codegen id : 24] +Output [6]: [sales#78, number_sales#79, catalog AS channel#80, i_brand_id#66, i_class_id#67, i_category_id#68] +Input [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sales#78, number_sales#79] (84) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] +Output [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#85)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (85) CometFilter -Input [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] -Condition : isnotnull(ws_item_sk#80) +Input [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] +Condition : isnotnull(ws_item_sk#81) -(86) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83] +(86) ColumnarToRow [codegen id : 35] +Input [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] (87) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(88) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#80] -Right keys [1]: [ss_item_sk#37] +(88) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#81] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (89) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] +Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(90) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#80] -Right keys [1]: [i_item_sk#85] +(90) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#81] +Right keys [1]: [i_item_sk#86] Join type: Inner Join condition: None -(91) Project [codegen id : 77] -Output [6]: [ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88] -Input [8]: [ws_item_sk#80, ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] +(91) Project [codegen id : 35] +Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [8]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(92) ReusedExchange [Reuses operator id: 129] -Output [1]: [d_date_sk#89] +(92) ReusedExchange [Reuses operator id: 127] +Output [1]: [d_date_sk#90] -(93) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#83] -Right keys [1]: [d_date_sk#89] +(93) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_sold_date_sk#84] +Right keys [1]: [d_date_sk#90] Join type: Inner Join condition: None -(94) Project [codegen id : 77] -Output [5]: [ws_quantity#81, ws_list_price#82, i_brand_id#86, i_class_id#87, i_category_id#88] -Input [7]: [ws_quantity#81, ws_list_price#82, ws_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, d_date_sk#89] +(94) Project [codegen id : 35] +Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_brand_id#87, i_class_id#88, i_category_id#89, d_date_sk#90] -(95) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#81, ws_list_price#82, i_brand_id#86, i_class_id#87, i_category_id#88] -Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] -Functions [2]: [partial_sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82)), partial_count(1)] -Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] -Results [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] +(95) HashAggregate [codegen id : 35] +Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#87, i_class_id#88, i_category_id#89] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [partial_sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83)), partial_count(1)] +Aggregate Attributes [3]: [sum#91, isEmpty#92, count#93] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#94, isEmpty#95, count#96] (96) Exchange -Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#94, isEmpty#95, count#96] +Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(97) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] -Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] -Functions [2]: [sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82))#96, count(1)#97] -Results [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum((cast(ws_quantity#81 as decimal(10,0)) * ws_list_price#82))#96 AS sales#98, count(1)#97 AS number_sales#99] +(97) HashAggregate [codegen id : 36] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#94, isEmpty#95, count#96] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83))#97, count(1)#98] +Results [5]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83))#97 AS sales#99, count(1)#98 AS number_sales#100] -(98) Filter [codegen id : 78] -Input [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sales#98, number_sales#99] -Condition : (isnotnull(sales#98) AND (cast(sales#98 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) +(98) Filter [codegen id : 36] +Input [5]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Condition : (isnotnull(sales#99) AND (cast(sales#99 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) -(99) Project [codegen id : 78] -Output [6]: [sales#98, number_sales#99, web AS channel#100, i_brand_id#86, i_class_id#87, i_category_id#88] -Input [5]: [i_brand_id#86, i_class_id#87, i_category_id#88, sales#98, number_sales#99] +(99) Project [codegen id : 36] +Output [6]: [sales#99, number_sales#100, web AS channel#101, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [5]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] (100) Union -(101) Expand [codegen id : 79] -Input [6]: [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58] -Arguments: [[sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58, 0], [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, null, 1], [sales#51, number_sales#52, channel#55, i_brand_id#56, null, null, 3], [sales#51, number_sales#52, channel#55, null, null, null, 7], [sales#51, number_sales#52, null, null, null, null, 15]], [sales#51, number_sales#52, channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] +(101) Expand [codegen id : 37] +Input [6]: [sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, i_category_id#59] +Arguments: [[sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, i_category_id#59, 0], [sales#52, number_sales#53, channel#56, i_brand_id#57, i_class_id#58, null, 1], [sales#52, number_sales#53, channel#56, i_brand_id#57, null, null, 3], [sales#52, number_sales#53, channel#56, null, null, null, 7], [sales#52, number_sales#53, null, null, null, null, 15]], [sales#52, number_sales#53, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] -(102) HashAggregate [codegen id : 79] -Input [7]: [sales#51, number_sales#52, channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] -Keys [5]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] -Functions [2]: [partial_sum(sales#51), partial_sum(number_sales#52)] -Aggregate Attributes [3]: [sum#106, isEmpty#107, sum#108] -Results [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] +(102) HashAggregate [codegen id : 37] +Input [7]: [sales#52, number_sales#53, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] +Aggregate Attributes [3]: [sum#107, isEmpty#108, sum#109] +Results [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] (103) Exchange -Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] -Arguments: hashpartitioning(channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] +Arguments: hashpartitioning(channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(104) HashAggregate [codegen id : 80] -Input [8]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105, sum#109, isEmpty#110, sum#111] -Keys [5]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, spark_grouping_id#105] -Functions [2]: [sum(sales#51), sum(number_sales#52)] -Aggregate Attributes [2]: [sum(sales#51)#112, sum(number_sales#52)#113] -Results [6]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales#51)#112 AS sum(sales)#114, sum(number_sales#52)#113 AS sum(number_sales)#115] +(104) HashAggregate [codegen id : 38] +Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] +Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#113, sum(number_sales#53)#114] +Results [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales#52)#113 AS sum(sales)#115, sum(number_sales#53)#114 AS sum(number_sales)#116] (105) TakeOrderedAndProject -Input [6]: [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales)#114, sum(number_sales)#115] -Arguments: 100, [channel#101 ASC NULLS FIRST, i_brand_id#102 ASC NULLS FIRST, i_class_id#103 ASC NULLS FIRST, i_category_id#104 ASC NULLS FIRST], [channel#101, i_brand_id#102, i_class_id#103, i_category_id#104, sum(sales)#114, sum(number_sales)#115] +Input [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#115, sum(number_sales)#116] +Arguments: 100, [channel#102 ASC NULLS FIRST, i_brand_id#103 ASC NULLS FIRST, i_class_id#104 ASC NULLS FIRST, i_category_id#105 ASC NULLS FIRST], [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#115, sum(number_sales)#116] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* HashAggregate (124) -+- Exchange (123) - +- * HashAggregate (122) - +- Union (121) - :- * Project (110) - : +- * BroadcastHashJoin Inner BuildRight (109) - : :- * ColumnarToRow (107) - : : +- CometScan parquet spark_catalog.default.store_sales (106) - : +- ReusedExchange (108) - :- * Project (115) - : +- * BroadcastHashJoin Inner BuildRight (114) - : :- * ColumnarToRow (112) - : : +- CometScan parquet spark_catalog.default.catalog_sales (111) - : +- ReusedExchange (113) - +- * Project (120) - +- * BroadcastHashJoin Inner BuildRight (119) - :- * ColumnarToRow (117) - : +- CometScan parquet spark_catalog.default.web_sales (116) - +- ReusedExchange (118) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (122) ++- Exchange (121) + +- * ColumnarToRow (120) + +- CometHashAggregate (119) + +- CometUnion (118) + :- CometProject (109) + : +- CometBroadcastHashJoin (108) + : :- CometScan parquet spark_catalog.default.store_sales (106) + : +- ReusedExchange (107) + :- CometProject (113) + : +- CometBroadcastHashJoin (112) + : :- CometScan parquet spark_catalog.default.catalog_sales (110) + : +- ReusedExchange (111) + +- CometProject (117) + +- CometBroadcastHashJoin (116) + :- CometScan parquet spark_catalog.default.web_sales (114) + +- ReusedExchange (115) (106) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#116, ss_list_price#117, ss_sold_date_sk#118] +Output [3]: [ss_quantity#117, ss_list_price#118, ss_sold_date_sk#119] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#118), dynamicpruningexpression(ss_sold_date_sk#118 IN dynamicpruning#119)] +PartitionFilters: [isnotnull(ss_sold_date_sk#119), dynamicpruningexpression(ss_sold_date_sk#119 IN dynamicpruning#120)] ReadSchema: struct -(107) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#116, ss_list_price#117, ss_sold_date_sk#118] - -(108) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#120] +(107) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#121] -(109) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#118] -Right keys [1]: [d_date_sk#120] -Join type: Inner -Join condition: None +(108) CometBroadcastHashJoin +Left output [3]: [ss_quantity#117, ss_list_price#118, ss_sold_date_sk#119] +Right output [1]: [d_date_sk#121] +Arguments: [ss_sold_date_sk#119], [d_date_sk#121], Inner, BuildRight -(110) Project [codegen id : 2] -Output [2]: [ss_quantity#116 AS quantity#121, ss_list_price#117 AS list_price#122] -Input [4]: [ss_quantity#116, ss_list_price#117, ss_sold_date_sk#118, d_date_sk#120] +(109) CometProject +Input [4]: [ss_quantity#117, ss_list_price#118, ss_sold_date_sk#119, d_date_sk#121] +Arguments: [quantity#122, list_price#123], [ss_quantity#117 AS quantity#122, ss_list_price#118 AS list_price#123] -(111) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#123, cs_list_price#124, cs_sold_date_sk#125] +(110) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#124, cs_list_price#125, cs_sold_date_sk#126] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#125), dynamicpruningexpression(cs_sold_date_sk#125 IN dynamicpruning#126)] +PartitionFilters: [isnotnull(cs_sold_date_sk#126), dynamicpruningexpression(cs_sold_date_sk#126 IN dynamicpruning#127)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#123, cs_list_price#124, cs_sold_date_sk#125] +(111) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#128] -(113) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#127] +(112) CometBroadcastHashJoin +Left output [3]: [cs_quantity#124, cs_list_price#125, cs_sold_date_sk#126] +Right output [1]: [d_date_sk#128] +Arguments: [cs_sold_date_sk#126], [d_date_sk#128], Inner, BuildRight -(114) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#125] -Right keys [1]: [d_date_sk#127] -Join type: Inner -Join condition: None +(113) CometProject +Input [4]: [cs_quantity#124, cs_list_price#125, cs_sold_date_sk#126, d_date_sk#128] +Arguments: [quantity#129, list_price#130], [cs_quantity#124 AS quantity#129, cs_list_price#125 AS list_price#130] -(115) Project [codegen id : 4] -Output [2]: [cs_quantity#123 AS quantity#128, cs_list_price#124 AS list_price#129] -Input [4]: [cs_quantity#123, cs_list_price#124, cs_sold_date_sk#125, d_date_sk#127] - -(116) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#130, ws_list_price#131, ws_sold_date_sk#132] +(114) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#131, ws_list_price#132, ws_sold_date_sk#133] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#132), dynamicpruningexpression(ws_sold_date_sk#132 IN dynamicpruning#133)] +PartitionFilters: [isnotnull(ws_sold_date_sk#133), dynamicpruningexpression(ws_sold_date_sk#133 IN dynamicpruning#134)] ReadSchema: struct -(117) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#130, ws_list_price#131, ws_sold_date_sk#132] +(115) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#135] -(118) ReusedExchange [Reuses operator id: 134] -Output [1]: [d_date_sk#134] +(116) CometBroadcastHashJoin +Left output [3]: [ws_quantity#131, ws_list_price#132, ws_sold_date_sk#133] +Right output [1]: [d_date_sk#135] +Arguments: [ws_sold_date_sk#133], [d_date_sk#135], Inner, BuildRight -(119) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#132] -Right keys [1]: [d_date_sk#134] -Join type: Inner -Join condition: None +(117) CometProject +Input [4]: [ws_quantity#131, ws_list_price#132, ws_sold_date_sk#133, d_date_sk#135] +Arguments: [quantity#136, list_price#137], [ws_quantity#131 AS quantity#136, ws_list_price#132 AS list_price#137] -(120) Project [codegen id : 6] -Output [2]: [ws_quantity#130 AS quantity#135, ws_list_price#131 AS list_price#136] -Input [4]: [ws_quantity#130, ws_list_price#131, ws_sold_date_sk#132, d_date_sk#134] +(118) CometUnion +Child 0 Input [2]: [quantity#122, list_price#123] +Child 1 Input [2]: [quantity#129, list_price#130] +Child 2 Input [2]: [quantity#136, list_price#137] -(121) Union - -(122) HashAggregate [codegen id : 7] -Input [2]: [quantity#121, list_price#122] +(119) CometHashAggregate +Input [2]: [quantity#122, list_price#123] Keys: [] -Functions [1]: [partial_avg((cast(quantity#121 as decimal(10,0)) * list_price#122))] -Aggregate Attributes [2]: [sum#137, count#138] -Results [2]: [sum#139, count#140] +Functions [1]: [partial_avg((cast(quantity#122 as decimal(10,0)) * list_price#123))] + +(120) ColumnarToRow [codegen id : 1] +Input [2]: [sum#138, count#139] -(123) Exchange -Input [2]: [sum#139, count#140] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +(121) Exchange +Input [2]: [sum#138, count#139] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(124) HashAggregate [codegen id : 8] -Input [2]: [sum#139, count#140] +(122) HashAggregate [codegen id : 2] +Input [2]: [sum#138, count#139] Keys: [] -Functions [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))] -Aggregate Attributes [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))#141] -Results [1]: [avg((cast(quantity#121 as decimal(10,0)) * list_price#122))#141 AS average_sales#142] +Functions [1]: [avg((cast(quantity#122 as decimal(10,0)) * list_price#123))] +Aggregate Attributes [1]: [avg((cast(quantity#122 as decimal(10,0)) * list_price#123))#140] +Results [1]: [avg((cast(quantity#122 as decimal(10,0)) * list_price#123))#140 AS average_sales#141] -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#118 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#119 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 111 Hosting Expression = cs_sold_date_sk#125 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#126 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 116 Hosting Expression = ws_sold_date_sk#132 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 114 Hosting Expression = ws_sold_date_sk#133 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (129) -+- * ColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan parquet spark_catalog.default.date_dim (125) +BroadcastExchange (127) ++- * ColumnarToRow (126) + +- CometProject (125) + +- CometFilter (124) + +- CometScan parquet spark_catalog.default.date_dim (123) -(125) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#42, d_year#143, d_moy#144] +(123) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#142, d_moy#143] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(126) CometFilter -Input [3]: [d_date_sk#42, d_year#143, d_moy#144] -Condition : ((((isnotnull(d_year#143) AND isnotnull(d_moy#144)) AND (d_year#143 = 2001)) AND (d_moy#144 = 11)) AND isnotnull(d_date_sk#42)) +(124) CometFilter +Input [3]: [d_date_sk#43, d_year#142, d_moy#143] +Condition : ((((isnotnull(d_year#142) AND isnotnull(d_moy#143)) AND (d_year#142 = 2001)) AND (d_moy#143 = 11)) AND isnotnull(d_date_sk#43)) -(127) CometProject -Input [3]: [d_date_sk#42, d_year#143, d_moy#144] -Arguments: [d_date_sk#42], [d_date_sk#42] +(125) CometProject +Input [3]: [d_date_sk#43, d_year#142, d_moy#143] +Arguments: [d_date_sk#43], [d_date_sk#43] -(128) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(126) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(129) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(127) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (134) -+- * ColumnarToRow (133) - +- CometProject (132) - +- CometFilter (131) - +- CometScan parquet spark_catalog.default.date_dim (130) +BroadcastExchange (132) ++- * ColumnarToRow (131) + +- CometProject (130) + +- CometFilter (129) + +- CometScan parquet spark_catalog.default.date_dim (128) -(130) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#145] +(128) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#144] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(131) CometFilter -Input [2]: [d_date_sk#25, d_year#145] -Condition : (((isnotnull(d_year#145) AND (d_year#145 >= 1999)) AND (d_year#145 <= 2001)) AND isnotnull(d_date_sk#25)) +(129) CometFilter +Input [2]: [d_date_sk#26, d_year#144] +Condition : (((isnotnull(d_year#144) AND (d_year#144 >= 1999)) AND (d_year#144 <= 2001)) AND isnotnull(d_date_sk#26)) -(132) CometProject -Input [2]: [d_date_sk#25, d_year#145] -Arguments: [d_date_sk#25], [d_date_sk#25] +(130) CometProject +Input [2]: [d_date_sk#26, d_year#144] +Arguments: [d_date_sk#26], [d_date_sk#26] -(133) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(131) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(134) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +(132) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#63 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] +Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] -Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index cf688c4486..e3dfa631b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -1,56 +1,45 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (80) + WholeStageCodegen (38) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (79) + WholeStageCodegen (37) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (26) + WholeStageCodegen (12) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #3 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #13 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] + Exchange #14 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 + CometHashAggregate [quantity,list_price] + CometUnion + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (25) + WholeStageCodegen (11) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -71,7 +60,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (11) + WholeStageCodegen (4) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow @@ -80,19 +69,19 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (10) + WholeStageCodegen (3) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 @@ -103,56 +92,43 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - ReusedExchange [d_date_sk] #7 + CometBroadcastExchange #8 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #9 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange #10 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #11 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #11 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #12 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - InputAdapter - ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + ReusedExchange [d_date_sk] #11 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (23) + BroadcastExchange #13 + WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter @@ -162,14 +138,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [ss_item_sk] #4 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (52) + WholeStageCodegen (24) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (51) + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (23) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -184,17 +160,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (78) + WholeStageCodegen (36) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (77) + Exchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (35) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -209,6 +185,6 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 7bba251deb..38ffcaa3a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -23,43 +23,43 @@ TakeOrderedAndProject (84) : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) : : : :- * HashAggregate (35) : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) + : : : : +- * ColumnarToRow (33) + : : : : +- CometHashAggregate (32) + : : : : +- CometProject (31) + : : : : +- CometBroadcastHashJoin (30) + : : : : :- CometProject (28) + : : : : : +- CometBroadcastHashJoin (27) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- CometBroadcastExchange (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (10) + : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometProject (23) + : : : : : +- CometBroadcastHashJoin (22) + : : : : : :- CometProject (17) + : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : :- CometFilter (12) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : +- CometBroadcastExchange (15) + : : : : : : +- CometFilter (14) + : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : +- CometBroadcastExchange (21) + : : : : : +- CometProject (20) + : : : : : +- CometFilter (19) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) + : : : : +- ReusedExchange (29) : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * ColumnarToRow (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) + : : : +- * ColumnarToRow (44) + : : : +- CometProject (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (38) + : : : +- ReusedExchange (41) : : +- BroadcastExchange (57) : : +- * BroadcastHashJoin LeftSemi BuildRight (56) : : :- * ColumnarToRow (54) @@ -97,7 +97,7 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 25] +(3) ColumnarToRow [codegen id : 11] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (4) Scan parquet spark_catalog.default.item @@ -111,7 +111,7 @@ ReadSchema: struct Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -151,605 +145,599 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(19) BroadcastExchange +(15) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(17) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#24] +(18) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(19) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(20) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] + +(21) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] + +(22) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight + +(23) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(24) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(25) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(28) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#25] +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(31) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] (34) Exchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] +Results [3]: [brand_id#27, class_id#28, category_id#29] (36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (37) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(42) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#36] +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] (45) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(48) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(49) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] (50) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [ss_item_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (52) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (53) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) -(54) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] (55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (57) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(58) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(59) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#42] +(60) ReusedExchange [Reuses operator id: 106] +Output [1]: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(62) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(63) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] (64) Exchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(65) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(65) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [6]: [store AS channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#53, count(1)#51 AS number_sales#54] -(66) Filter [codegen id : 52] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(66) Filter [codegen id : 24] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Condition : (isnotnull(sales#53) AND (cast(sales#53 as decimal(32,6)) > cast(Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (67) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (68) CometFilter -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] -Condition : isnotnull(ss_item_sk#56) +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +Condition : isnotnull(ss_item_sk#57) -(69) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +(69) ColumnarToRow [codegen id : 22] +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] (70) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(71) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [ss_item_sk#37] +(71) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (72) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [i_item_sk#61] +(73) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [i_item_sk#62] Join type: Inner Join condition: None -(74) Project [codegen id : 50] -Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +(74) Project [codegen id : 22] +Output [6]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(75) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#65] +(75) ReusedExchange [Reuses operator id: 120] +Output [1]: [d_date_sk#66] -(76) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#65] +(76) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_sold_date_sk#60] +Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(77) Project [codegen id : 50] -Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] +(77) Project [codegen id : 22] +Output [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] -(78) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] -Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +(78) HashAggregate [codegen id : 22] +Input [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] (79) Exchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(80) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72, count(1)#73] -Results [6]: [store AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] +(80) HashAggregate [codegen id : 23] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#73, count(1)#74] +Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#73 AS sales#76, count(1)#74 AS number_sales#77] -(81) Filter [codegen id : 51] -Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(81) Filter [codegen id : 23] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (82) BroadcastExchange -Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=8] -(83) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Right keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +(83) BroadcastHashJoin [codegen id : 24] +Left keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Join type: Inner Join condition: None (84) TakeOrderedAndProject -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] +Input [12]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (103) -+- Exchange (102) - +- * HashAggregate (101) - +- Union (100) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * ColumnarToRow (86) - : : +- CometScan parquet spark_catalog.default.store_sales (85) - : +- ReusedExchange (87) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * ColumnarToRow (91) - : : +- CometScan parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (92) - +- * Project (99) - +- * BroadcastHashJoin Inner BuildRight (98) - :- * ColumnarToRow (96) - : +- CometScan parquet spark_catalog.default.web_sales (95) - +- ReusedExchange (97) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#55, [id=#56] +* HashAggregate (101) ++- Exchange (100) + +- * ColumnarToRow (99) + +- CometHashAggregate (98) + +- CometUnion (97) + :- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometScan parquet spark_catalog.default.store_sales (85) + : +- ReusedExchange (86) + :- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometScan parquet spark_catalog.default.catalog_sales (89) + : +- ReusedExchange (90) + +- CometProject (96) + +- CometBroadcastHashJoin (95) + :- CometScan parquet spark_catalog.default.web_sales (93) + +- ReusedExchange (94) (85) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] +Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#79), dynamicpruningexpression(ss_sold_date_sk#79 IN dynamicpruning#80)] +PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] - -(87) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#81] +(86) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#82] -(88) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#79] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None +(87) CometBroadcastHashJoin +Left output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] +Right output [1]: [d_date_sk#82] +Arguments: [ss_sold_date_sk#80], [d_date_sk#82], Inner, BuildRight -(89) Project [codegen id : 2] -Output [2]: [ss_quantity#77 AS quantity#82, ss_list_price#78 AS list_price#83] -Input [4]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79, d_date_sk#81] +(88) CometProject +Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] +Arguments: [quantity#83, list_price#84], [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] -(90) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] +(89) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#86), dynamicpruningexpression(cs_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] ReadSchema: struct -(91) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] +(90) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#89] -(92) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#88] +(91) CometBroadcastHashJoin +Left output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +Right output [1]: [d_date_sk#89] +Arguments: [cs_sold_date_sk#87], [d_date_sk#89], Inner, BuildRight -(93) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#86] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None +(92) CometProject +Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] +Arguments: [quantity#90, list_price#91], [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] -(94) Project [codegen id : 4] -Output [2]: [cs_quantity#84 AS quantity#89, cs_list_price#85 AS list_price#90] -Input [4]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86, d_date_sk#88] - -(95) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +(93) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] ReadSchema: struct -(96) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +(94) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#96] -(97) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#95] +(95) CometBroadcastHashJoin +Left output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Right output [1]: [d_date_sk#96] +Arguments: [ws_sold_date_sk#94], [d_date_sk#96], Inner, BuildRight -(98) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#93] -Right keys [1]: [d_date_sk#95] -Join type: Inner -Join condition: None +(96) CometProject +Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] +Arguments: [quantity#97, list_price#98], [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] -(99) Project [codegen id : 6] -Output [2]: [ws_quantity#91 AS quantity#96, ws_list_price#92 AS list_price#97] -Input [4]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93, d_date_sk#95] +(97) CometUnion +Child 0 Input [2]: [quantity#83, list_price#84] +Child 1 Input [2]: [quantity#90, list_price#91] +Child 2 Input [2]: [quantity#97, list_price#98] -(100) Union - -(101) HashAggregate [codegen id : 7] -Input [2]: [quantity#82, list_price#83] +(98) CometHashAggregate +Input [2]: [quantity#83, list_price#84] Keys: [] -Functions [1]: [partial_avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] -Aggregate Attributes [2]: [sum#98, count#99] -Results [2]: [sum#100, count#101] +Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] + +(99) ColumnarToRow [codegen id : 1] +Input [2]: [sum#99, count#100] -(102) Exchange -Input [2]: [sum#100, count#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(100) Exchange +Input [2]: [sum#99, count#100] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(103) HashAggregate [codegen id : 8] -Input [2]: [sum#100, count#101] +(101) HashAggregate [codegen id : 2] +Input [2]: [sum#99, count#100] Keys: [] -Functions [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] -Aggregate Attributes [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102] -Results [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102 AS average_sales#103] +Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#101] +Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#101 AS average_sales#102] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#79 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * ColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan parquet spark_catalog.default.date_dim (104) +BroadcastExchange (106) ++- * ColumnarToRow (105) + +- CometProject (104) + +- CometFilter (103) + +- CometScan parquet spark_catalog.default.date_dim (102) -(104) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#42, d_week_seq#104] +(102) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(105) CometFilter -Input [2]: [d_date_sk#42, d_week_seq#104] -Condition : ((isnotnull(d_week_seq#104) AND (d_week_seq#104 = Subquery scalar-subquery#105, [id=#106])) AND isnotnull(d_date_sk#42)) +(103) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#103] +Condition : ((isnotnull(d_week_seq#103) AND (d_week_seq#103 = Subquery scalar-subquery#104, [id=#105])) AND isnotnull(d_date_sk#43)) -(106) CometProject -Input [2]: [d_date_sk#42, d_week_seq#104] -Arguments: [d_date_sk#42], [d_date_sk#42] +(104) CometProject +Input [2]: [d_date_sk#43, d_week_seq#103] +Arguments: [d_date_sk#43], [d_date_sk#43] -(107) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(105) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(108) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(106) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:6 Hosting operator id = 105 Hosting Expression = Subquery scalar-subquery#105, [id=#106] -* ColumnarToRow (112) -+- CometProject (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +Subquery:6 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#104, [id=#105] +* ColumnarToRow (110) ++- CometProject (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(109) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] +(107) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(110) CometFilter -Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] -Condition : (((((isnotnull(d_year#108) AND isnotnull(d_moy#109)) AND isnotnull(d_dom#110)) AND (d_year#108 = 2000)) AND (d_moy#109 = 12)) AND (d_dom#110 = 11)) +(108) CometFilter +Input [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] +Condition : (((((isnotnull(d_year#107) AND isnotnull(d_moy#108)) AND isnotnull(d_dom#109)) AND (d_year#107 = 2000)) AND (d_moy#108 = 12)) AND (d_dom#109 = 11)) -(111) CometProject -Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] -Arguments: [d_week_seq#107], [d_week_seq#107] +(109) CometProject +Input [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] +Arguments: [d_week_seq#106], [d_week_seq#106] -(112) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#107] +(110) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#106] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (115) ++- * ColumnarToRow (114) + +- CometProject (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) -(113) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#111] +(111) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#110] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter -Input [2]: [d_date_sk#25, d_year#111] -Condition : (((isnotnull(d_year#111) AND (d_year#111 >= 1999)) AND (d_year#111 <= 2001)) AND isnotnull(d_date_sk#25)) +(112) CometFilter +Input [2]: [d_date_sk#26, d_year#110] +Condition : (((isnotnull(d_year#110) AND (d_year#110 >= 1999)) AND (d_year#110 <= 2001)) AND isnotnull(d_date_sk#26)) -(115) CometProject -Input [2]: [d_date_sk#25, d_year#111] -Arguments: [d_date_sk#25], [d_date_sk#25] +(113) CometProject +Input [2]: [d_date_sk#26, d_year#110] +Arguments: [d_date_sk#26], [d_date_sk#26] -(116) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(114) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(117) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(115) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 -BroadcastExchange (122) -+- * ColumnarToRow (121) - +- CometProject (120) - +- CometFilter (119) - +- CometScan parquet spark_catalog.default.date_dim (118) +Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#60 IN dynamicpruning#61 +BroadcastExchange (120) ++- * ColumnarToRow (119) + +- CometProject (118) + +- CometFilter (117) + +- CometScan parquet spark_catalog.default.date_dim (116) -(118) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#112] +(116) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#66, d_week_seq#111] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(119) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#112] -Condition : ((isnotnull(d_week_seq#112) AND (d_week_seq#112 = Subquery scalar-subquery#113, [id=#114])) AND isnotnull(d_date_sk#65)) +(117) CometFilter +Input [2]: [d_date_sk#66, d_week_seq#111] +Condition : ((isnotnull(d_week_seq#111) AND (d_week_seq#111 = Subquery scalar-subquery#112, [id=#113])) AND isnotnull(d_date_sk#66)) -(120) CometProject -Input [2]: [d_date_sk#65, d_week_seq#112] -Arguments: [d_date_sk#65], [d_date_sk#65] +(118) CometProject +Input [2]: [d_date_sk#66, d_week_seq#111] +Arguments: [d_date_sk#66], [d_date_sk#66] -(121) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] +(119) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#66] -(122) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +(120) BroadcastExchange +Input [1]: [d_date_sk#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:12 Hosting operator id = 119 Hosting Expression = Subquery scalar-subquery#113, [id=#114] -* ColumnarToRow (126) -+- CometProject (125) - +- CometFilter (124) - +- CometScan parquet spark_catalog.default.date_dim (123) +Subquery:12 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#112, [id=#113] +* ColumnarToRow (124) ++- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) -(123) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] +(121) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(124) CometFilter -Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] -Condition : (((((isnotnull(d_year#116) AND isnotnull(d_moy#117)) AND isnotnull(d_dom#118)) AND (d_year#116 = 1999)) AND (d_moy#117 = 12)) AND (d_dom#118 = 11)) +(122) CometFilter +Input [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] +Condition : (((((isnotnull(d_year#115) AND isnotnull(d_moy#116)) AND isnotnull(d_dom#117)) AND (d_year#115 = 1999)) AND (d_moy#116 = 12)) AND (d_dom#117 = 11)) -(125) CometProject -Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] -Arguments: [d_week_seq#115], [d_week_seq#115] +(123) CometProject +Input [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] +Arguments: [d_week_seq#114], [d_week_seq#114] -(126) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#115] +(124) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#114] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 09d8d9dde3..799f74a369 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -1,47 +1,36 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (24) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] + Exchange #13 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 + CometHashAggregate [quantity,list_price] + CometUnion + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (11) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -69,7 +58,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #3 - WholeStageCodegen (11) + WholeStageCodegen (4) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow @@ -78,19 +67,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (10) + WholeStageCodegen (3) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 @@ -101,56 +90,43 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 + CometBroadcastExchange #7 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange #9 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #10 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #11 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) + BroadcastExchange #12 + WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter @@ -161,14 +137,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) + BroadcastExchange #14 + WholeStageCodegen (23) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (22) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -180,7 +156,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 + BroadcastExchange #16 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -197,6 +173,6 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter - ReusedExchange [d_date_sk] #15 + ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt index 13a621f773..c4772ea809 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt @@ -1,26 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.customer (4) - : +- BroadcastExchange (13) - : +- * ColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.customer_address (10) - +- ReusedExchange (16) +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- Exchange (21) + +- * ColumnarToRow (20) + +- CometHashAggregate (19) + +- CometProject (18) + +- CometBroadcastHashJoin (17) + :- CometProject (12) + : +- CometBroadcastHashJoin (11) + : :- CometProject (7) + : : +- CometBroadcastHashJoin (6) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : +- CometBroadcastExchange (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.customer (3) + : +- CometBroadcastExchange (10) + : +- CometFilter (9) + : +- CometScan parquet spark_catalog.default.customer_address (8) + +- CometBroadcastExchange (16) + +- CometProject (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.date_dim (13) (1) Scan parquet spark_catalog.default.catalog_sales @@ -35,130 +36,135 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_bill_customer_sk#1) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.customer +(3) Scan parquet spark_catalog.default.customer Output [2]: [c_customer_sk#5, c_current_addr_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [c_customer_sk#5, c_current_addr_sk#6] Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) -(6) ColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [c_customer_sk#5, c_current_addr_sk#6] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] +Right output [2]: [c_customer_sk#5, c_current_addr_sk#6] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#5], Inner, BuildRight -(9) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] +(7) CometProject Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] +Arguments: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6], [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] -(10) Scan parquet spark_catalog.default.customer_address +(8) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] Condition : isnotnull(ca_address_sk#7) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] +Arguments: [ca_address_sk#7, ca_state#8, ca_zip#9] -(13) BroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_current_addr_sk#6] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: ((substr(ca_zip#9, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#8 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) +(11) CometBroadcastHashJoin +Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] +Right output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] +Arguments: [c_current_addr_sk#6], [ca_address_sk#7], Inner, ((substr(ca_zip#9, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#8 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)), BuildRight -(15) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9] +(12) CometProject Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#8, ca_zip#9] +Arguments: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9], [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9] + +(13) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_qoy#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(14) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] +Condition : ((((isnotnull(d_qoy#12) AND isnotnull(d_year#11)) AND (d_qoy#12 = 2)) AND (d_year#11 = 2001)) AND isnotnull(d_date_sk#10)) -(16) ReusedExchange [Reuses operator id: 27] -Output [1]: [d_date_sk#10] +(15) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(16) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9] +Right output [1]: [d_date_sk#10] +Arguments: [cs_sold_date_sk#3], [d_date_sk#10], Inner, BuildRight -(18) Project [codegen id : 4] -Output [2]: [cs_sales_price#2, ca_zip#9] +(18) CometProject Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9, d_date_sk#10] +Arguments: [cs_sales_price#2, ca_zip#9], [cs_sales_price#2, ca_zip#9] -(19) HashAggregate [codegen id : 4] +(19) CometHashAggregate Input [2]: [cs_sales_price#2, ca_zip#9] Keys [1]: [ca_zip#9] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum#11] -Results [2]: [ca_zip#9, sum#12] -(20) Exchange -Input [2]: [ca_zip#9, sum#12] -Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(20) ColumnarToRow [codegen id : 1] +Input [2]: [ca_zip#9, sum#13] + +(21) Exchange +Input [2]: [ca_zip#9, sum#13] +Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#9, sum#12] +(22) HashAggregate [codegen id : 2] +Input [2]: [ca_zip#9, sum#13] Keys [1]: [ca_zip#9] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#13] -Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#13,17,2) AS sum(cs_sales_price)#14] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#14] +Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#14,17,2) AS sum(cs_sales_price)#15] -(22) TakeOrderedAndProject -Input [2]: [ca_zip#9, sum(cs_sales_price)#14] -Arguments: 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#14] +(23) TakeOrderedAndProject +Input [2]: [ca_zip#9, sum(cs_sales_price)#15] +Arguments: 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#15] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) -(23) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#15, d_qoy#16] +(24) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_qoy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] -Condition : ((((isnotnull(d_qoy#16) AND isnotnull(d_year#15)) AND (d_qoy#16 = 2)) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#10)) +(25) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] +Condition : ((((isnotnull(d_qoy#12) AND isnotnull(d_year#11)) AND (d_qoy#12 = 2)) AND (d_year#11 = 2001)) AND isnotnull(d_date_sk#10)) -(25) CometProject -Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] +(26) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_qoy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(26) ColumnarToRow [codegen id : 1] +(27) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(27) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt index 5c750b2db8..a033463726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] InputAdapter Exchange [ca_zip] #1 - WholeStageCodegen (4) - HashAggregate [ca_zip,cs_sales_price] [sum,sum] - Project [cs_sales_price,ca_zip] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] - Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_zip,cs_sales_price] + CometProject [cs_sales_price,ca_zip] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + CometProject [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -23,19 +23,13 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] CometProject [d_date_sk] CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #3 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #4 + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index 0de98cfb0f..8d9edc0a04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -1,44 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * ColumnarToRow (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometProject (18) + : : : : +- CometBroadcastHashJoin (17) + : : : : :- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (8) + : : : : +- CometBroadcastExchange (16) + : : : : +- CometProject (15) + : : : : +- CometFilter (14) + : : : : +- CometScan parquet spark_catalog.default.date_dim (13) + : : : +- CometBroadcastExchange (22) + : : : +- CometProject (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.date_dim (19) + : : +- ReusedExchange (25) + : +- CometBroadcastExchange (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.store (28) + +- CometBroadcastExchange (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.item (33) (1) Scan parquet spark_catalog.default.store_sales @@ -53,10 +55,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) -(6) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] +Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] +Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] +(7) CometProject Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -(10) Scan parquet spark_catalog.default.catalog_sales +(8) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] @@ -93,206 +88,224 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] +Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +(12) CometProject Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#19] +(13) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#19, d_quarter_name#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None +(14) CometFilter +Input [2]: [d_date_sk#19, d_quarter_name#20] +Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND isnotnull(d_date_sk#19)) -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +(15) CometProject +Input [2]: [d_date_sk#19, d_quarter_name#20] +Arguments: [d_date_sk#19], [d_date_sk#19] + +(16) CometBroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: [d_date_sk#19] + +(17) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight + +(18) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] + +(19) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#21, d_quarter_name#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(20) CometFilter +Input [2]: [d_date_sk#21, d_quarter_name#22] +Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) + +(21) CometProject +Input [2]: [d_date_sk#21, d_quarter_name#22] +Arguments: [d_date_sk#21], [d_date_sk#21] -(19) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#20] +(22) CometBroadcastExchange +Input [1]: [d_date_sk#21] +Arguments: [d_date_sk#21] -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#21] +Arguments: [sr_returned_date_sk#12], [d_date_sk#21], Inner, BuildRight -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] +(24) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -(22) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#21] +(25) ReusedExchange [Reuses operator id: 22] +Output [1]: [d_date_sk#23] -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None +(26) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#23] +Arguments: [cs_sold_date_sk#17], [d_date_sk#23], Inner, BuildRight -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] +(27) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#23] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -(25) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_state#23] +(28) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(26) CometFilter -Input [2]: [s_store_sk#22, s_state#23] -Condition : isnotnull(s_store_sk#22) +(29) CometFilter +Input [2]: [s_store_sk#24, s_state#25] +Condition : isnotnull(s_store_sk#24) -(27) ColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#22, s_state#23] +(30) CometBroadcastExchange +Input [2]: [s_store_sk#24, s_state#25] +Arguments: [s_store_sk#24, s_state#25] -(28) BroadcastExchange -Input [2]: [s_store_sk#22, s_state#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(31) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] +Right output [2]: [s_store_sk#24, s_state#25] +Arguments: [ss_store_sk#3], [s_store_sk#24], Inner, BuildRight -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None +(32) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#24, s_state#25] +Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25] -(30) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_state#23] - -(31) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] +(33) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Condition : isnotnull(i_item_sk#24) - -(33) ColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] +(34) CometFilter +Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] +Condition : isnotnull(i_item_sk#26) -(34) BroadcastExchange -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(35) CometBroadcastExchange +Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] +Arguments: [i_item_sk#26, i_item_id#27, i_item_desc#28] -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#24] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25] +Right output [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] +Arguments: [ss_item_sk#1], [i_item_sk#26], Inner, BuildRight -(36) Project [codegen id : 8] -Output [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#25, i_item_desc#26] -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_sk#24, i_item_id#25, i_item_desc#26] +(37) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_sk#26, i_item_id#27, i_item_desc#28] +Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28] -(37) HashAggregate [codegen id : 8] -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#25, i_item_desc#26] -Keys [3]: [i_item_id#25, i_item_desc#26, s_state#23] +(38) CometHashAggregate +Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28] +Keys [3]: [i_item_id#27, i_item_desc#28, s_state#25] Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [18]: [count#27, sum#28, count#29, n#30, avg#31, m2#32, count#33, sum#34, count#35, n#36, avg#37, m2#38, count#39, sum#40, count#41, n#42, avg#43, m2#44] -Results [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] -(38) Exchange -Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] -Arguments: hashpartitioning(i_item_id#25, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(39) ColumnarToRow [codegen id : 1] +Input [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#29, sum#30, count#31, n#32, avg#33, m2#34, count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46] + +(40) Exchange +Input [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#29, sum#30, count#31, n#32, avg#33, m2#34, count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46] +Arguments: hashpartitioning(i_item_id#27, i_item_desc#28, s_state#25, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(39) HashAggregate [codegen id : 9] -Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] -Keys [3]: [i_item_id#25, i_item_desc#26, s_state#23] +(41) HashAggregate [codegen id : 2] +Input [21]: [i_item_id#27, i_item_desc#28, s_state#25, count#29, sum#30, count#31, n#32, avg#33, m2#34, count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46] +Keys [3]: [i_item_id#27, i_item_desc#28, s_state#25] Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [9]: [count(ss_quantity#5)#63, avg(ss_quantity#5)#64, stddev_samp(cast(ss_quantity#5 as double))#65, count(sr_return_quantity#11)#66, avg(sr_return_quantity#11)#67, stddev_samp(cast(sr_return_quantity#11 as double))#68, count(cs_quantity#16)#69, avg(cs_quantity#16)#70, stddev_samp(cast(cs_quantity#16 as double))#71] -Results [15]: [i_item_id#25, i_item_desc#26, s_state#23, count(ss_quantity#5)#63 AS store_sales_quantitycount#72, avg(ss_quantity#5)#64 AS store_sales_quantityave#73, stddev_samp(cast(ss_quantity#5 as double))#65 AS store_sales_quantitystdev#74, (stddev_samp(cast(ss_quantity#5 as double))#65 / avg(ss_quantity#5)#64) AS store_sales_quantitycov#75, count(sr_return_quantity#11)#66 AS as_store_returns_quantitycount#76, avg(sr_return_quantity#11)#67 AS as_store_returns_quantityave#77, stddev_samp(cast(sr_return_quantity#11 as double))#68 AS as_store_returns_quantitystdev#78, (stddev_samp(cast(sr_return_quantity#11 as double))#68 / avg(sr_return_quantity#11)#67) AS store_returns_quantitycov#79, count(cs_quantity#16)#69 AS catalog_sales_quantitycount#80, avg(cs_quantity#16)#70 AS catalog_sales_quantityave#81, (stddev_samp(cast(cs_quantity#16 as double))#71 / avg(cs_quantity#16)#70) AS catalog_sales_quantitystdev#82, (stddev_samp(cast(cs_quantity#16 as double))#71 / avg(cs_quantity#16)#70) AS catalog_sales_quantitycov#83] +Aggregate Attributes [9]: [count(ss_quantity#5)#47, avg(ss_quantity#5)#48, stddev_samp(cast(ss_quantity#5 as double))#49, count(sr_return_quantity#11)#50, avg(sr_return_quantity#11)#51, stddev_samp(cast(sr_return_quantity#11 as double))#52, count(cs_quantity#16)#53, avg(cs_quantity#16)#54, stddev_samp(cast(cs_quantity#16 as double))#55] +Results [15]: [i_item_id#27, i_item_desc#28, s_state#25, count(ss_quantity#5)#47 AS store_sales_quantitycount#56, avg(ss_quantity#5)#48 AS store_sales_quantityave#57, stddev_samp(cast(ss_quantity#5 as double))#49 AS store_sales_quantitystdev#58, (stddev_samp(cast(ss_quantity#5 as double))#49 / avg(ss_quantity#5)#48) AS store_sales_quantitycov#59, count(sr_return_quantity#11)#50 AS as_store_returns_quantitycount#60, avg(sr_return_quantity#11)#51 AS as_store_returns_quantityave#61, stddev_samp(cast(sr_return_quantity#11 as double))#52 AS as_store_returns_quantitystdev#62, (stddev_samp(cast(sr_return_quantity#11 as double))#52 / avg(sr_return_quantity#11)#51) AS store_returns_quantitycov#63, count(cs_quantity#16)#53 AS catalog_sales_quantitycount#64, avg(cs_quantity#16)#54 AS catalog_sales_quantityave#65, (stddev_samp(cast(cs_quantity#16 as double))#55 / avg(cs_quantity#16)#54) AS catalog_sales_quantitystdev#66, (stddev_samp(cast(cs_quantity#16 as double))#55 / avg(cs_quantity#16)#54) AS catalog_sales_quantitycov#67] -(40) TakeOrderedAndProject -Input [15]: [i_item_id#25, i_item_desc#26, s_state#23, store_sales_quantitycount#72, store_sales_quantityave#73, store_sales_quantitystdev#74, store_sales_quantitycov#75, as_store_returns_quantitycount#76, as_store_returns_quantityave#77, as_store_returns_quantitystdev#78, store_returns_quantitycov#79, catalog_sales_quantitycount#80, catalog_sales_quantityave#81, catalog_sales_quantitystdev#82, catalog_sales_quantitycov#83] -Arguments: 100, [i_item_id#25 ASC NULLS FIRST, i_item_desc#26 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#25, i_item_desc#26, s_state#23, store_sales_quantitycount#72, store_sales_quantityave#73, store_sales_quantitystdev#74, store_sales_quantitycov#75, as_store_returns_quantitycount#76, as_store_returns_quantityave#77, as_store_returns_quantitystdev#78, store_returns_quantitycov#79, catalog_sales_quantitycount#80, catalog_sales_quantityave#81, catalog_sales_quantitystdev#82, catalog_sales_quantitycov#83] +(42) TakeOrderedAndProject +Input [15]: [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#56, store_sales_quantityave#57, store_sales_quantitystdev#58, store_sales_quantitycov#59, as_store_returns_quantitycount#60, as_store_returns_quantityave#61, as_store_returns_quantitystdev#62, store_returns_quantitycov#63, catalog_sales_quantitycount#64, catalog_sales_quantityave#65, catalog_sales_quantitystdev#66, catalog_sales_quantitycov#67] +Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_state#25 ASC NULLS FIRST], [i_item_id#27, i_item_desc#28, s_state#25, store_sales_quantitycount#56, store_sales_quantityave#57, store_sales_quantitystdev#58, store_sales_quantitycov#59, as_store_returns_quantitycount#60, as_store_returns_quantityave#61, as_store_returns_quantitystdev#62, store_returns_quantitycov#63, catalog_sales_quantitycount#64, catalog_sales_quantityave#65, catalog_sales_quantitystdev#66, catalog_sales_quantitycov#67] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(41) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#84] +(43) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#19, d_quarter_name#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#84] -Condition : ((isnotnull(d_quarter_name#84) AND (d_quarter_name#84 = 2001Q1)) AND isnotnull(d_date_sk#19)) +(44) CometFilter +Input [2]: [d_date_sk#19, d_quarter_name#20] +Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND isnotnull(d_date_sk#19)) -(43) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#84] +(45) CometProject +Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(44) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(45) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_quarter_name#85] +(48) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#21, d_quarter_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [2]: [d_date_sk#20, d_quarter_name#85] -Condition : (d_quarter_name#85 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#20)) +(49) CometFilter +Input [2]: [d_date_sk#21, d_quarter_name#22] +Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) -(48) CometProject -Input [2]: [d_date_sk#20, d_quarter_name#85] -Arguments: [d_date_sk#20], [d_date_sk#20] +(50) CometProject +Input [2]: [d_date_sk#21, d_quarter_name#22] +Arguments: [d_date_sk#21], [d_date_sk#21] -(49) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] +(51) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#21] -(50) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(52) BroadcastExchange +Input [1]: [d_date_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index 9f4d67decc..77aba376e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - WholeStageCodegen (9) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] InputAdapter Exchange [i_item_id,i_item_desc,s_state] #1 - WholeStageCodegen (8) - 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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] + CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,46 +31,33 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s CometProject [d_date_sk] CometFilter [d_quarter_name,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_quarter_name,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometBroadcastExchange #3 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometBroadcastExchange #5 + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometBroadcastExchange #7 + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange #8 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange #9 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt index 1e9c660c55..c09ef14458 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt @@ -1,47 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Expand (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Project (17) - : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * ColumnarToRow (7) - : : : : : +- CometProject (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : : +- BroadcastExchange (15) - : : : : +- * ColumnarToRow (14) - : : : : +- CometProject (13) - : : : : +- CometFilter (12) - : : : : +- CometScan parquet spark_catalog.default.customer (11) - : : : +- BroadcastExchange (21) - : : : +- * ColumnarToRow (20) - : : : +- CometFilter (19) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) - : : +- BroadcastExchange (27) - : : +- * ColumnarToRow (26) - : : +- CometFilter (25) - : : +- CometScan parquet spark_catalog.default.customer_address (24) - : +- ReusedExchange (30) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.item (33) +TakeOrderedAndProject (41) ++- * HashAggregate (40) + +- Exchange (39) + +- * ColumnarToRow (38) + +- CometHashAggregate (37) + +- CometExpand (36) + +- CometProject (35) + +- CometBroadcastHashJoin (34) + :- CometProject (30) + : +- CometBroadcastHashJoin (29) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometProject (19) + : : : +- CometBroadcastHashJoin (18) + : : : :- CometProject (14) + : : : : +- CometBroadcastHashJoin (13) + : : : : :- CometProject (8) + : : : : : +- CometBroadcastHashJoin (7) + : : : : : :- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : +- CometBroadcastExchange (6) + : : : : : +- CometProject (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : : : +- CometBroadcastExchange (12) + : : : : +- CometProject (11) + : : : : +- CometFilter (10) + : : : : +- CometScan parquet spark_catalog.default.customer (9) + : : : +- CometBroadcastExchange (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (15) + : : +- CometBroadcastExchange (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.customer_address (20) + : +- CometBroadcastExchange (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.date_dim (25) + +- CometBroadcastExchange (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.item (31) (1) Scan parquet spark_catalog.default.catalog_sales @@ -56,226 +54,219 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) ColumnarToRow [codegen id : 1] -Input [2]: [cd_demo_sk#11, cd_dep_count#14] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#11, cd_dep_count#14] -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Right output [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +(8) CometProject Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(11) Scan parquet spark_catalog.default.customer +(9) Scan parquet spark_catalog.default.customer Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) -(13) CometProject +(11) CometProject Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) ColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) BroadcastExchange +(12) CometBroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +(14) CometProject Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(18) Scan parquet spark_catalog.default.customer_demographics +(15) Scan parquet spark_catalog.default.customer_demographics Output [1]: [cd_demo_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(19) CometFilter +(16) CometFilter Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) ColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#20] - -(21) BroadcastExchange +(17) CometBroadcastExchange Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [cd_demo_sk#20] -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [cd_demo_sk#20] +Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +(19) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(24) Scan parquet spark_catalog.default.customer_address +(20) Scan parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [IN,MS,ND,NM,OK,VA]), IsNotNull(ca_address_sk)] ReadSchema: struct -(25) CometFilter +(21) CometFilter Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) -(26) ColumnarToRow [codegen id : 4] +(22) CometBroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -(27) BroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -(28) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight -(29) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +(24) CometProject Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] -(30) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#25] +(25) Scan parquet spark_catalog.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,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(31) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(26) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : ((isnotnull(d_year#26) AND (d_year#26 = 1998)) AND isnotnull(d_date_sk#25)) -(32) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +(27) CometProject +Input [2]: [d_date_sk#25, d_year#26] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(28) CometBroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: [d_date_sk#25] + +(29) CometBroadcastHashJoin +Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#9], [d_date_sk#25], Inner, BuildRight + +(30) CometProject Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, d_date_sk#25] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] -(33) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#26, i_item_id#27] +(31) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#27, i_item_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) CometFilter -Input [2]: [i_item_sk#26, i_item_id#27] -Condition : isnotnull(i_item_sk#26) - -(35) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#26, i_item_id#27] +(32) CometFilter +Input [2]: [i_item_sk#27, i_item_id#28] +Condition : isnotnull(i_item_sk#27) -(36) BroadcastExchange -Input [2]: [i_item_sk#26, i_item_id#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(33) CometBroadcastExchange +Input [2]: [i_item_sk#27, i_item_id#28] +Arguments: [i_item_sk#27, i_item_id#28] -(37) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None +(34) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Right output [2]: [i_item_sk#27, i_item_id#28] +Arguments: [cs_item_sk#3], [i_item_sk#27], Inner, BuildRight -(38) Project [codegen id : 7] -Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, ca_county#22] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#26, i_item_id#27] +(35) CometProject +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#27, i_item_id#28] +Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22] -(39) Expand [codegen id : 7] -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, ca_county#22] -Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] +(36) CometExpand +Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22] +Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] -(40) HashAggregate [codegen id : 7] -Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] -Keys [5]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] +(37) CometHashAggregate +Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] +Keys [5]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] -Aggregate Attributes [14]: [sum#33, count#34, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46] -Results [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] -(41) Exchange -Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] -Arguments: hashpartitioning(i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(38) ColumnarToRow [codegen id : 1] +Input [19]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47] -(42) HashAggregate [codegen id : 8] -Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] -Keys [5]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] +(39) Exchange +Input [19]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47] +Arguments: hashpartitioning(i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(40) HashAggregate [codegen id : 2] +Input [19]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33, sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47] +Keys [5]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] -Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#61, avg(cast(cs_list_price#5 as decimal(12,2)))#62, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63, avg(cast(cs_sales_price#6 as decimal(12,2)))#64, avg(cast(cs_net_profit#8 as decimal(12,2)))#65, avg(cast(c_birth_year#19 as decimal(12,2)))#66, avg(cast(cd_dep_count#14 as decimal(12,2)))#67] -Results [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, avg(cast(cs_quantity#4 as decimal(12,2)))#61 AS agg1#68, avg(cast(cs_list_price#5 as decimal(12,2)))#62 AS agg2#69, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63 AS agg3#70, avg(cast(cs_sales_price#6 as decimal(12,2)))#64 AS agg4#71, avg(cast(cs_net_profit#8 as decimal(12,2)))#65 AS agg5#72, avg(cast(c_birth_year#19 as decimal(12,2)))#66 AS agg6#73, avg(cast(cd_dep_count#14 as decimal(12,2)))#67 AS agg7#74] +Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#48, avg(cast(cs_list_price#5 as decimal(12,2)))#49, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#50, avg(cast(cs_sales_price#6 as decimal(12,2)))#51, avg(cast(cs_net_profit#8 as decimal(12,2)))#52, avg(cast(c_birth_year#19 as decimal(12,2)))#53, avg(cast(cd_dep_count#14 as decimal(12,2)))#54] +Results [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, avg(cast(cs_quantity#4 as decimal(12,2)))#48 AS agg1#55, avg(cast(cs_list_price#5 as decimal(12,2)))#49 AS agg2#56, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#50 AS agg3#57, avg(cast(cs_sales_price#6 as decimal(12,2)))#51 AS agg4#58, avg(cast(cs_net_profit#8 as decimal(12,2)))#52 AS agg5#59, avg(cast(c_birth_year#19 as decimal(12,2)))#53 AS agg6#60, avg(cast(cd_dep_count#14 as decimal(12,2)))#54 AS agg7#61] -(43) TakeOrderedAndProject -Input [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] -Arguments: 100, [ca_country#29 ASC NULLS FIRST, ca_state#30 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] +(41) TakeOrderedAndProject +Input [11]: [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#55, agg2#56, agg3#57, agg4#58, agg5#59, agg6#60, agg7#61] +Arguments: 100, [ca_country#30 ASC NULLS FIRST, ca_state#31 ASC NULLS FIRST, ca_county#32 ASC NULLS FIRST, i_item_id#29 ASC NULLS FIRST], [i_item_id#29, ca_country#30, ca_state#31, ca_county#32, agg1#55, agg2#56, agg3#57, agg4#58, agg5#59, agg6#60, agg7#61] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) -(44) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#75] +(42) Scan parquet spark_catalog.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,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [2]: [d_date_sk#25, d_year#75] -Condition : ((isnotnull(d_year#75) AND (d_year#75 = 1998)) AND isnotnull(d_date_sk#25)) +(43) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : ((isnotnull(d_year#26) AND (d_year#26 = 1998)) AND isnotnull(d_date_sk#25)) -(46) CometProject -Input [2]: [d_date_sk#25, d_year#75] +(44) CometProject +Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(47) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(48) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt index 47911b9ba3..3d101857bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] + CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -30,42 +30,24 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange #5 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange #6 + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange #7 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #8 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt index 999fec8383..a00474beec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt @@ -1,43 +1,38 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (17) - : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.date_dim (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometFilter (6) - : : : : +- CometScan parquet spark_catalog.default.store_sales (5) - : : : +- BroadcastExchange (15) - : : : +- * ColumnarToRow (14) - : : : +- CometProject (13) - : : : +- CometFilter (12) - : : : +- CometScan parquet spark_catalog.default.item (11) - : : +- BroadcastExchange (21) - : : +- * ColumnarToRow (20) - : : +- CometFilter (19) - : : +- CometScan parquet spark_catalog.default.customer (18) - : +- BroadcastExchange (27) - : +- * ColumnarToRow (26) - : +- CometFilter (25) - : +- CometScan parquet spark_catalog.default.customer_address (24) - +- BroadcastExchange (33) - +- * ColumnarToRow (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.store (30) +TakeOrderedAndProject (34) ++- * HashAggregate (33) + +- Exchange (32) + +- * ColumnarToRow (31) + +- CometHashAggregate (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (14) + : : : +- CometBroadcastHashJoin (13) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (12) + : : : +- CometProject (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.item (9) + : : +- CometBroadcastExchange (17) + : : +- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.customer (15) + : +- CometBroadcastExchange (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.customer_address (20) + +- CometBroadcastExchange (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.store (25) (1) Scan parquet spark_catalog.default.date_dim @@ -55,10 +50,7 @@ Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) Input [3]: [d_date_sk#1, d_year#2, d_moy#3] Arguments: [d_date_sk#1], [d_date_sk#1] -(4) ColumnarToRow [codegen id : 6] -Input [1]: [d_date_sk#1] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -66,162 +58,143 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) -(7) ColumnarToRow [codegen id : 1] -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[4, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#8] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [1]: [d_date_sk#1] +Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] +Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight -(10) Project [codegen id : 6] -Output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] +(8) CometProject Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] +Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] -(11) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] 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 -(12) CometFilter +(10) CometFilter Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) -(13) CometProject +(11) CometProject Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] Arguments: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Arguments: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -(15) BroadcastExchange -Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#4] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] +Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight -(17) Project [codegen id : 6] -Output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +(14) CometProject Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -(18) Scan parquet spark_catalog.default.customer +(15) Scan parquet spark_catalog.default.customer Output [2]: [c_customer_sk#15, c_current_addr_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(19) CometFilter +(16) CometFilter Input [2]: [c_customer_sk#15, c_current_addr_sk#16] Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) -(20) ColumnarToRow [codegen id : 3] -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] - -(21) BroadcastExchange +(17) CometBroadcastExchange Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [c_customer_sk#15, c_current_addr_sk#16] -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#5] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Right output [2]: [c_customer_sk#15, c_current_addr_sk#16] +Arguments: [ss_customer_sk#5], [c_customer_sk#15], Inner, BuildRight -(23) Project [codegen id : 6] -Output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16] +(19) CometProject Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_customer_sk#15, c_current_addr_sk#16] +Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16] -(24) Scan parquet spark_catalog.default.customer_address +(20) Scan parquet spark_catalog.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 -(25) CometFilter +(21) CometFilter Input [2]: [ca_address_sk#17, ca_zip#18] Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_zip#18)) -(26) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#17, ca_zip#18] - -(27) BroadcastExchange +(22) CometBroadcastExchange Input [2]: [ca_address_sk#17, ca_zip#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: [ca_address_sk#17, ca_zip#18] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_addr_sk#16] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16] +Right output [2]: [ca_address_sk#17, ca_zip#18] +Arguments: [c_current_addr_sk#16], [ca_address_sk#17], Inner, BuildRight -(29) Project [codegen id : 6] -Output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18] +(24) CometProject Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16, ca_address_sk#17, ca_zip#18] +Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18] -(30) Scan parquet spark_catalog.default.store +(25) Scan parquet spark_catalog.default.store Output [2]: [s_store_sk#19, s_zip#20] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_zip), IsNotNull(s_store_sk)] ReadSchema: struct -(31) CometFilter +(26) CometFilter Input [2]: [s_store_sk#19, s_zip#20] Condition : (isnotnull(s_zip#20) AND isnotnull(s_store_sk#19)) -(32) ColumnarToRow [codegen id : 5] +(27) CometBroadcastExchange Input [2]: [s_store_sk#19, s_zip#20] +Arguments: [s_store_sk#19, s_zip#20] -(33) BroadcastExchange -Input [2]: [s_store_sk#19, s_zip#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -(34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#19] -Join type: Inner -Join condition: NOT (substr(ca_zip#18, 1, 5) = substr(s_zip#20, 1, 5)) +(28) CometBroadcastHashJoin +Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18] +Right output [2]: [s_store_sk#19, s_zip#20] +Arguments: [ss_store_sk#6], [s_store_sk#19], Inner, NOT (substr(ca_zip#18, 1, 5) = substr(s_zip#20, 1, 5)), BuildRight -(35) Project [codegen id : 6] -Output [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +(29) CometProject Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18, s_store_sk#19, s_zip#20] +Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -(36) HashAggregate [codegen id : 6] +(30) CometHashAggregate Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] Keys [4]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] -Aggregate Attributes [1]: [sum#21] -Results [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] -(37) Exchange -Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] -Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(31) ColumnarToRow [codegen id : 1] +Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#21] + +(32) Exchange +Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#21] +Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(38) HashAggregate [codegen id : 7] -Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] +(33) HashAggregate [codegen id : 2] +Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#21] Keys [4]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#23] -Results [5]: [i_brand_id#10 AS brand_id#24, i_brand#11 AS brand#25, i_manufact_id#12, i_manufact#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#23,17,2) AS ext_price#26] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#22] +Results [5]: [i_brand_id#10 AS brand_id#23, i_brand#11 AS brand#24, i_manufact_id#12, i_manufact#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#22,17,2) AS ext_price#25] -(39) TakeOrderedAndProject -Input [5]: [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] -Arguments: 100, [ext_price#26 DESC NULLS LAST, brand#25 ASC NULLS FIRST, brand_id#24 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] +(34) TakeOrderedAndProject +Input [5]: [brand_id#23, brand#24, i_manufact_id#12, i_manufact#13, ext_price#25] +Arguments: 100, [ext_price#25 DESC NULLS LAST, brand#24 ASC NULLS FIRST, brand_id#23 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#23, brand#24, i_manufact_id#12, i_manufact#13, ext_price#25] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt index c2f5d1a876..18a69bcb4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt @@ -1,58 +1,38 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] - WholeStageCodegen (7) + WholeStageCodegen (2) HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - WholeStageCodegen (6) - 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] - BroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] - Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [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] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometBroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_zip] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [s_zip,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] + CometBroadcastExchange #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometBroadcastExchange #4 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #5 + CometFilter [ca_address_sk,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange #6 + CometFilter [s_zip,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt index 18de3145cc..5c6b73c4c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt @@ -1,40 +1,39 @@ == Physical Plan == -* Sort (36) -+- Exchange (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * ColumnarToRow (6) - : : : +- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.date_dim (7) - : +- BroadcastExchange (20) - : +- * ColumnarToRow (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.date_dim (16) - +- BroadcastExchange (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * HashAggregate (24) - : +- ReusedExchange (23) - +- BroadcastExchange (29) - +- * ColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.date_dim (25) +* Sort (35) ++- Exchange (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * HashAggregate (14) + : : +- Exchange (13) + : : +- * ColumnarToRow (12) + : : +- CometHashAggregate (11) + : : +- CometProject (10) + : : +- CometBroadcastHashJoin (9) + : : :- CometUnion (5) + : : : :- CometProject (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- CometProject (4) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) + : : +- CometBroadcastExchange (8) + : : +- CometFilter (7) + : : +- CometScan parquet spark_catalog.default.date_dim (6) + : +- BroadcastExchange (19) + : +- * ColumnarToRow (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.date_dim (15) + +- BroadcastExchange (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * HashAggregate (23) + : +- ReusedExchange (22) + +- BroadcastExchange (28) + +- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) (1) Scan parquet spark_catalog.default.web_sales @@ -63,148 +62,142 @@ Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7 Child 0 Input [2]: [sold_date_sk#3, sales_price#4] Child 1 Input [2]: [sold_date_sk#7, sales_price#8] -(6) ColumnarToRow [codegen id : 2] -Input [2]: [sold_date_sk#3, sales_price#4] - -(7) Scan parquet spark_catalog.default.date_dim +(6) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] ReadSchema: struct -(8) CometFilter +(7) CometFilter Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) -(9) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] - -(10) BroadcastExchange +(8) CometBroadcastExchange Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#11] -(11) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [sold_date_sk#3] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometBroadcastHashJoin +Left output [2]: [sold_date_sk#3, sales_price#4] +Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] +Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight -(12) Project [codegen id : 2] -Output [3]: [sales_price#4, d_week_seq#10, d_day_name#11] +(10) CometProject Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#11] +Arguments: [sales_price#4, d_week_seq#10, d_day_name#11], [sales_price#4, d_week_seq#10, d_day_name#11] -(13) HashAggregate [codegen id : 2] +(11) CometHashAggregate Input [3]: [sales_price#4, d_week_seq#10, d_day_name#11] Keys [1]: [d_week_seq#10] Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18] -Results [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] -(14) Exchange -Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(12) ColumnarToRow [codegen id : 1] +Input [8]: [d_week_seq#10, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18] + +(13) Exchange +Input [8]: [d_week_seq#10, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18] +Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 8] -Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] +(14) HashAggregate [codegen id : 6] +Input [8]: [d_week_seq#10, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26,17,2) AS sun_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27,17,2) AS mon_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28,17,2) AS tue_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29,17,2) AS wed_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30,17,2) AS thu_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31,17,2) AS fri_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32,17,2) AS sat_sales#39] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#25] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#19,17,2) AS sun_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#20,17,2) AS mon_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#21,17,2) AS tue_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#22,17,2) AS wed_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#23,17,2) AS thu_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#24,17,2) AS fri_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#25,17,2) AS sat_sales#32] -(16) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#40, d_year#41] +(15) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#33, d_year#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] ReadSchema: struct -(17) CometFilter -Input [2]: [d_week_seq#40, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2001)) AND isnotnull(d_week_seq#40)) +(16) CometFilter +Input [2]: [d_week_seq#33, d_year#34] +Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2001)) AND isnotnull(d_week_seq#33)) -(18) CometProject -Input [2]: [d_week_seq#40, d_year#41] -Arguments: [d_week_seq#40], [d_week_seq#40] +(17) CometProject +Input [2]: [d_week_seq#33, d_year#34] +Arguments: [d_week_seq#33], [d_week_seq#33] -(19) ColumnarToRow [codegen id : 3] -Input [1]: [d_week_seq#40] +(18) ColumnarToRow [codegen id : 2] +Input [1]: [d_week_seq#33] -(20) BroadcastExchange -Input [1]: [d_week_seq#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(19) BroadcastExchange +Input [1]: [d_week_seq#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(21) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#40] +Right keys [1]: [d_week_seq#33] Join type: Inner Join condition: None -(22) Project [codegen id : 8] -Output [8]: [d_week_seq#10 AS d_week_seq1#42, sun_sales#33 AS sun_sales1#43, mon_sales#34 AS mon_sales1#44, tue_sales#35 AS tue_sales1#45, wed_sales#36 AS wed_sales1#46, thu_sales#37 AS thu_sales1#47, fri_sales#38 AS fri_sales1#48, sat_sales#39 AS sat_sales1#49] -Input [9]: [d_week_seq#10, sun_sales#33, mon_sales#34, tue_sales#35, wed_sales#36, thu_sales#37, fri_sales#38, sat_sales#39, d_week_seq#40] +(21) Project [codegen id : 6] +Output [8]: [d_week_seq#10 AS d_week_seq1#35, sun_sales#26 AS sun_sales1#36, mon_sales#27 AS mon_sales1#37, tue_sales#28 AS tue_sales1#38, wed_sales#29 AS wed_sales1#39, thu_sales#30 AS thu_sales1#40, fri_sales#31 AS fri_sales1#41, sat_sales#32 AS sat_sales1#42] +Input [9]: [d_week_seq#10, sun_sales#26, mon_sales#27, tue_sales#28, wed_sales#29, thu_sales#30, fri_sales#31, sat_sales#32, d_week_seq#33] -(23) ReusedExchange [Reuses operator id: 14] -Output [8]: [d_week_seq#10, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] +(22) ReusedExchange [Reuses operator id: 13] +Output [8]: [d_week_seq#10, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] -(24) HashAggregate [codegen id : 7] -Input [8]: [d_week_seq#10, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] +(23) HashAggregate [codegen id : 5] +Input [8]: [d_week_seq#10, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26,17,2) AS sun_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27,17,2) AS mon_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28,17,2) AS tue_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29,17,2) AS wed_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30,17,2) AS thu_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31,17,2) AS fri_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32,17,2) AS sat_sales#39] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#25] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#19,17,2) AS sun_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#20,17,2) AS mon_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#21,17,2) AS tue_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#22,17,2) AS wed_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#23,17,2) AS thu_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#24,17,2) AS fri_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#25,17,2) AS sat_sales#32] -(25) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#57, d_year#58] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#50, d_year#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct -(26) CometFilter -Input [2]: [d_week_seq#57, d_year#58] -Condition : ((isnotnull(d_year#58) AND (d_year#58 = 2002)) AND isnotnull(d_week_seq#57)) +(25) CometFilter +Input [2]: [d_week_seq#50, d_year#51] +Condition : ((isnotnull(d_year#51) AND (d_year#51 = 2002)) AND isnotnull(d_week_seq#50)) -(27) CometProject -Input [2]: [d_week_seq#57, d_year#58] -Arguments: [d_week_seq#57], [d_week_seq#57] +(26) CometProject +Input [2]: [d_week_seq#50, d_year#51] +Arguments: [d_week_seq#50], [d_week_seq#50] -(28) ColumnarToRow [codegen id : 6] -Input [1]: [d_week_seq#57] +(27) ColumnarToRow [codegen id : 4] +Input [1]: [d_week_seq#50] -(29) BroadcastExchange -Input [1]: [d_week_seq#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(28) BroadcastExchange +Input [1]: [d_week_seq#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 7] +(29) BroadcastHashJoin [codegen id : 5] Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#57] +Right keys [1]: [d_week_seq#50] Join type: Inner Join condition: None -(31) Project [codegen id : 7] -Output [8]: [d_week_seq#10 AS d_week_seq2#59, sun_sales#33 AS sun_sales2#60, mon_sales#34 AS mon_sales2#61, tue_sales#35 AS tue_sales2#62, wed_sales#36 AS wed_sales2#63, thu_sales#37 AS thu_sales2#64, fri_sales#38 AS fri_sales2#65, sat_sales#39 AS sat_sales2#66] -Input [9]: [d_week_seq#10, sun_sales#33, mon_sales#34, tue_sales#35, wed_sales#36, thu_sales#37, fri_sales#38, sat_sales#39, d_week_seq#57] +(30) Project [codegen id : 5] +Output [8]: [d_week_seq#10 AS d_week_seq2#52, sun_sales#26 AS sun_sales2#53, mon_sales#27 AS mon_sales2#54, tue_sales#28 AS tue_sales2#55, wed_sales#29 AS wed_sales2#56, thu_sales#30 AS thu_sales2#57, fri_sales#31 AS fri_sales2#58, sat_sales#32 AS sat_sales2#59] +Input [9]: [d_week_seq#10, sun_sales#26, mon_sales#27, tue_sales#28, wed_sales#29, thu_sales#30, fri_sales#31, sat_sales#32, d_week_seq#50] -(32) BroadcastExchange -Input [8]: [d_week_seq2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=5] +(31) BroadcastExchange +Input [8]: [d_week_seq2#52, sun_sales2#53, mon_sales2#54, tue_sales2#55, wed_sales2#56, thu_sales2#57, fri_sales2#58, sat_sales2#59] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=4] -(33) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [d_week_seq1#42] -Right keys [1]: [(d_week_seq2#59 - 53)] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [d_week_seq1#35] +Right keys [1]: [(d_week_seq2#52 - 53)] Join type: Inner Join condition: None -(34) Project [codegen id : 8] -Output [8]: [d_week_seq1#42, round((sun_sales1#43 / sun_sales2#60), 2) AS round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1#44 / mon_sales2#61), 2) AS round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1#45 / tue_sales2#62), 2) AS round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1#46 / wed_sales2#63), 2) AS round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1#47 / thu_sales2#64), 2) AS round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1#48 / fri_sales2#65), 2) AS round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1#49 / sat_sales2#66), 2) AS round((sat_sales1 / sat_sales2), 2)#73] -Input [16]: [d_week_seq1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66] +(33) Project [codegen id : 6] +Output [8]: [d_week_seq1#35, round((sun_sales1#36 / sun_sales2#53), 2) AS round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1#37 / mon_sales2#54), 2) AS round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1#38 / tue_sales2#55), 2) AS round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1#39 / wed_sales2#56), 2) AS round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1#40 / thu_sales2#57), 2) AS round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1#41 / fri_sales2#58), 2) AS round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1#42 / sat_sales2#59), 2) AS round((sat_sales1 / sat_sales2), 2)#66] +Input [16]: [d_week_seq1#35, sun_sales1#36, mon_sales1#37, tue_sales1#38, wed_sales1#39, thu_sales1#40, fri_sales1#41, sat_sales1#42, d_week_seq2#52, sun_sales2#53, mon_sales2#54, tue_sales2#55, wed_sales2#56, thu_sales2#57, fri_sales2#58, sat_sales2#59] -(35) Exchange -Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] -Arguments: rangepartitioning(d_week_seq1#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(34) Exchange +Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66] +Arguments: rangepartitioning(d_week_seq1#35 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(36) Sort [codegen id : 9] -Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1 / fri_sales2), 2)#72, round((sat_sales1 / sat_sales2), 2)#73] -Arguments: [d_week_seq1#42 ASC NULLS FIRST], true, 0 +(35) Sort [codegen id : 7] +Input [8]: [d_week_seq1#35, round((sun_sales1 / sun_sales2), 2)#60, round((mon_sales1 / mon_sales2), 2)#61, round((tue_sales1 / tue_sales2), 2)#62, round((wed_sales1 / wed_sales2), 2)#63, round((thu_sales1 / thu_sales2), 2)#64, round((fri_sales1 / fri_sales2), 2)#65, round((sat_sales1 / sat_sales2), 2)#66] +Arguments: [d_week_seq1#35 ASC NULLS FIRST], true, 0 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index 8856ce80d2..c7999d981e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (9) +WholeStageCodegen (7) Sort [d_week_seq1] InputAdapter Exchange [d_week_seq1] #1 - WholeStageCodegen (8) + WholeStageCodegen (6) Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] BroadcastHashJoin [d_week_seq1,d_week_seq2] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] @@ -10,27 +10,23 @@ WholeStageCodegen (9) HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq] #2 - WholeStageCodegen (2) - HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_week_seq,d_day_name,sales_price] + CometProject [sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [sold_date_sk,d_date_sk] CometUnion CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange #3 + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) + WholeStageCodegen (2) ColumnarToRow InputAdapter CometProject [d_week_seq] @@ -38,7 +34,7 @@ WholeStageCodegen (9) CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) + WholeStageCodegen (5) Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] @@ -46,7 +42,7 @@ WholeStageCodegen (9) ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) + WholeStageCodegen (4) ColumnarToRow InputAdapter CometProject [d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index 333ef218c6..7c29ff2183 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * ColumnarToRow (15) + +- CometHashAggregate (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.catalog_sales @@ -33,118 +35,127 @@ ReadSchema: struct Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [cs_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) CometHashAggregate Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(15) ColumnarToRow [codegen id : 1] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] -(16) Exchange +(18) Exchange Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] +(19) Sort [codegen id : 3] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window +(20) Window Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(19) Project [codegen id : 6] +(21) Project [codegen id : 4] Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] -(20) TakeOrderedAndProject +(22) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) -(21) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(23) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index 52c42bdf2b..d805e38682 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,12 +29,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index 77f17c7f36..21c979264b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -3,26 +3,26 @@ TakeOrderedAndProject (24) +- * Filter (23) +- * HashAggregate (22) +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.warehouse (4) - : +- BroadcastExchange (14) - : +- * ColumnarToRow (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.item (10) - +- ReusedExchange (17) + +- * ColumnarToRow (20) + +- CometHashAggregate (19) + +- CometProject (18) + +- CometBroadcastHashJoin (17) + :- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (7) + : : +- CometBroadcastHashJoin (6) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- CometBroadcastExchange (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.warehouse (3) + : +- CometBroadcastExchange (11) + : +- CometProject (10) + : +- CometFilter (9) + : +- CometScan parquet spark_catalog.default.item (8) + +- CometBroadcastExchange (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.date_dim (14) (1) Scan parquet spark_catalog.default.inventory @@ -37,107 +37,108 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Condition : isnotnull(w_warehouse_sk#6) -(6) ColumnarToRow [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [w_warehouse_sk#6, w_warehouse_name#7] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#6] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#6], Inner, BuildRight -(9) Project [codegen id : 4] -Output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] +(7) CometProject Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7], [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] -(10) Scan parquet spark_catalog.default.item +(8) Scan parquet spark_catalog.default.item Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) -(12) CometProject +(10) CometProject Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] Arguments: [i_item_sk#8, i_item_id#9], [i_item_sk#8, i_item_id#9] -(13) ColumnarToRow [codegen id : 2] +(11) CometBroadcastExchange Input [2]: [i_item_sk#8, i_item_id#9] +Arguments: [i_item_sk#8, i_item_id#9] -(14) BroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] +Right output [2]: [i_item_sk#8, i_item_id#9] +Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight -(16) Project [codegen id : 4] -Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9] +(13) CometProject Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#9] +Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9], [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9] -(17) ReusedExchange [Reuses operator id: 28] +(14) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] +ReadSchema: struct + +(15) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-10)) AND (d_date#12 <= 2000-04-10)) AND isnotnull(d_date_sk#11)) + +(16) CometBroadcastExchange +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11, d_date#12] -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9] +Right output [2]: [d_date_sk#11, d_date#12] +Arguments: [inv_date_sk#4], [d_date_sk#11], Inner, BuildRight -(19) Project [codegen id : 4] -Output [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] +(18) CometProject Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9, d_date_sk#11, d_date#12] +Arguments: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12], [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] -(20) HashAggregate [codegen id : 4] +(19) CometHashAggregate Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] Keys [2]: [w_warehouse_name#7, i_item_id#9] Functions [2]: [partial_sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] -Aggregate Attributes [2]: [sum#13, sum#14] -Results [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] + +(20) ColumnarToRow [codegen id : 1] +Input [4]: [w_warehouse_name#7, i_item_id#9, sum#13, sum#14] (21) Exchange -Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Input [4]: [w_warehouse_name#7, i_item_id#9, sum#13, sum#14] +Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] +(22) HashAggregate [codegen id : 2] +Input [4]: [w_warehouse_name#7, i_item_id#9, sum#13, sum#14] Keys [2]: [w_warehouse_name#7, i_item_id#9] Functions [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] -Aggregate Attributes [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18] -Results [4]: [w_warehouse_name#7, i_item_id#9, sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17 AS inv_before#19, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_after#20] +Aggregate Attributes [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#15, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#16] +Results [4]: [w_warehouse_name#7, i_item_id#9, sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#15 AS inv_before#17, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#16 AS inv_after#18] -(23) Filter [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] -Condition : (CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667) END AND CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5) END) +(23) Filter [codegen id : 2] +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18] +Condition : (CASE WHEN (inv_before#17 > 0) THEN ((cast(inv_after#18 as double) / cast(inv_before#17 as double)) >= 0.666667) END AND CASE WHEN (inv_before#17 > 0) THEN ((cast(inv_after#18 as double) / cast(inv_before#17 as double)) <= 1.5) END) (24) TakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] -Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18] +Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#17, inv_after#18] ===== Subqueries ===== @@ -164,6 +165,6 @@ Input [2]: [d_date_sk#11, d_date#12] (28) BroadcastExchange Input [2]: [d_date_sk#11, d_date#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt index e20755e12f..52bd7a85e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - WholeStageCodegen (5) + WholeStageCodegen (2) Filter [inv_before,inv_after] HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] InputAdapter Exchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometProject [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] CometFilter [inv_warehouse_sk,inv_item_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -23,20 +23,13 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] InputAdapter CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id] - CometFilter [i_current_price,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #2 + CometBroadcastExchange #3 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #4 + CometProject [i_item_sk,i_item_id] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometBroadcastExchange #5 + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt index 9f5771fed2..df6a80179e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt @@ -1,27 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.warehouse (13) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- Exchange (22) + +- * ColumnarToRow (21) + +- CometHashAggregate (20) + +- CometExpand (19) + +- CometProject (18) + +- CometBroadcastHashJoin (17) + :- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.item (9) + +- CometBroadcastExchange (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.warehouse (14) (1) Scan parquet spark_catalog.default.inventory @@ -36,134 +37,139 @@ ReadSchema: struct + +(4) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) -(4) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#6] +(5) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +(8) CometProject Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] +Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -(7) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(9) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) +(10) CometFilter +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Condition : isnotnull(i_item_sk#8) -(9) ColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(11) CometBroadcastExchange +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(10) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +Right output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(13) CometProject +Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(12) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] - -(13) Scan parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#12] +(14) Scan parquet spark_catalog.default.warehouse +Output [1]: [w_warehouse_sk#13] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(14) CometFilter -Input [1]: [w_warehouse_sk#12] -Condition : isnotnull(w_warehouse_sk#12) - -(15) ColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#12] +(15) CometFilter +Input [1]: [w_warehouse_sk#13] +Condition : isnotnull(w_warehouse_sk#13) -(16) BroadcastExchange -Input [1]: [w_warehouse_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastExchange +Input [1]: [w_warehouse_sk#13] +Arguments: [w_warehouse_sk#13] -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#12] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Right output [1]: [w_warehouse_sk#13] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#13], Inner, BuildRight -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11, w_warehouse_sk#12] +(18) CometProject +Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12, w_warehouse_sk#13] +Arguments: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11] -(19) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10] -Arguments: [[inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10, 0], [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, null, 1], [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, null, null, 3], [inv_quantity_on_hand#3, i_product_name#11, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] +(19) CometExpand +Input [5]: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11] +Arguments: [[inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11, 0], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, null, 1], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, null, null, 3], [inv_quantity_on_hand#3, i_product_name#12, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] -(20) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#3, i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] -Keys [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] +(20) CometHashAggregate +Input [6]: [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] +Keys [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#18, count#19] -Results [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] -(21) Exchange -Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] -Arguments: hashpartitioning(i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) ColumnarToRow [codegen id : 1] +Input [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#19, count#20] + +(22) Exchange +Input [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#19, count#20] +Arguments: hashpartitioning(i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] -Keys [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] +(23) HashAggregate [codegen id : 2] +Input [7]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18, sum#19, count#20] +Keys [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#22] -Results [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, avg(inv_quantity_on_hand#3)#22 AS qoh#23] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] +Results [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, avg(inv_quantity_on_hand#3)#21 AS qoh#22] -(23) TakeOrderedAndProject -Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] -Arguments: 100, [qoh#23 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] +(24) TakeOrderedAndProject +Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#22] +Arguments: 100, [qoh#22 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_category#17 ASC NULLS FIRST], [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * ColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.date_dim (24) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) -(24) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#24] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#24] -Condition : (((isnotnull(d_month_seq#24) AND (d_month_seq#24 >= 1200)) AND (d_month_seq#24 <= 1211)) AND isnotnull(d_date_sk#6)) +(26) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) -(26) CometProject -Input [2]: [d_date_sk#6, d_month_seq#24] +(27) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(27) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(28) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt index 92714bb02d..bda583c175 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] + CometExpand [i_product_name,i_brand,i_class,i_category] [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category,spark_grouping_id] + CometProject [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -24,19 +24,13 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index 78d2b85199..560a05db9a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -1,70 +1,71 @@ == Physical Plan == -* HashAggregate (66) -+- Exchange (65) - +- * HashAggregate (64) - +- Union (63) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * SortMergeJoin LeftSemi (41) - : : :- * Sort (24) - : : : +- Exchange (23) - : : : +- * Project (22) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (21) +* HashAggregate (67) ++- Exchange (66) + +- * HashAggregate (65) + +- Union (64) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (43) + : : +- * SortMergeJoin LeftSemi (42) + : : :- * Sort (26) + : : : +- Exchange (25) + : : : +- * Project (24) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (23) : : : :- * ColumnarToRow (2) : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (20) - : : : +- * Project (19) - : : : +- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- Exchange (16) - : : : +- * HashAggregate (15) - : : : +- * Project (14) - : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : :- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (12) - : : : +- * ColumnarToRow (11) - : : : +- CometFilter (10) - : : : +- CometScan parquet spark_catalog.default.item (9) - : : +- * Sort (40) - : : +- * Project (39) - : : +- * Filter (38) - : : +- * HashAggregate (37) - : : +- Exchange (36) - : : +- * HashAggregate (35) - : : +- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * ColumnarToRow (28) - : : : +- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometScan parquet spark_catalog.default.store_sales (25) - : : +- BroadcastExchange (32) - : : +- * ColumnarToRow (31) - : : +- CometFilter (30) - : : +- CometScan parquet spark_catalog.default.customer (29) - : +- ReusedExchange (43) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * SortMergeJoin LeftSemi (58) - : :- * Sort (52) - : : +- Exchange (51) - : : +- * Project (50) - : : +- * BroadcastHashJoin LeftSemi BuildRight (49) - : : :- * ColumnarToRow (47) - : : : +- CometScan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (48) - : +- * Sort (57) - : +- * Project (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- ReusedExchange (53) - +- ReusedExchange (60) + : : : +- BroadcastExchange (22) + : : : +- * Project (21) + : : : +- * Filter (20) + : : : +- * HashAggregate (19) + : : : +- Exchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometHashAggregate (16) + : : : +- CometProject (15) + : : : +- CometBroadcastHashJoin (14) + : : : :- CometProject (10) + : : : : +- CometBroadcastHashJoin (9) + : : : : :- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (8) + : : : : +- CometProject (7) + : : : : +- CometFilter (6) + : : : : +- CometScan parquet spark_catalog.default.date_dim (5) + : : : +- CometBroadcastExchange (13) + : : : +- CometFilter (12) + : : : +- CometScan parquet spark_catalog.default.item (11) + : : +- * Sort (41) + : : +- * Project (40) + : : +- * Filter (39) + : : +- * HashAggregate (38) + : : +- Exchange (37) + : : +- * ColumnarToRow (36) + : : +- CometHashAggregate (35) + : : +- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan parquet spark_catalog.default.store_sales (27) + : : +- CometBroadcastExchange (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.customer (30) + : +- ReusedExchange (44) + +- * Project (63) + +- * BroadcastHashJoin Inner BuildRight (62) + :- * Project (60) + : +- * SortMergeJoin LeftSemi (59) + : :- * Sort (53) + : : +- Exchange (52) + : : +- * Project (51) + : : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : : :- * ColumnarToRow (48) + : : : +- CometScan parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (49) + : +- * Sort (58) + : +- * Project (57) + : +- * Filter (56) + : +- * HashAggregate (55) + : +- ReusedExchange (54) + +- ReusedExchange (61) (1) Scan parquet spark_catalog.default.catalog_sales @@ -74,7 +75,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 3] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (3) Scan parquet spark_catalog.default.store_sales @@ -89,482 +90,500 @@ ReadSchema: struct Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(5) ColumnarToRow [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] +(5) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_date#11, d_year#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 76] -Output [2]: [d_date_sk#10, d_date#11] +(6) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(7) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(7) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] + +(8) CometBroadcastExchange +Input [2]: [d_date_sk#10, d_date#11] +Arguments: [d_date_sk#10, d_date#11] -(8) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#11] +(9) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] +Right output [2]: [d_date_sk#10, d_date#11] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight + +(10) CometProject Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] +Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] -(9) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#12, i_item_desc#13] +(11) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#13, i_item_desc#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) +(12) CometFilter +Input [2]: [i_item_sk#13, i_item_desc#14] +Condition : isnotnull(i_item_sk#13) -(11) ColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] +(13) CometBroadcastExchange +Input [2]: [i_item_sk#13, i_item_desc#14] +Arguments: [i_item_sk#13, i_item_desc#14] -(12) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(14) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#7, d_date#11] +Right output [2]: [i_item_sk#13, i_item_desc#14] +Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join type: Inner -Join condition: None +(15) CometProject +Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] +Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] -(14) Project [codegen id : 3] -Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] - -(15) HashAggregate [codegen id : 3] -Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +(16) CometHashAggregate +Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] +Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(16) Exchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(17) ColumnarToRow [codegen id : 1] +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] + +(18) Exchange +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] +Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(17) HashAggregate [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +(19) HashAggregate [codegen id : 2] +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] +Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] +Results [2]: [i_item_sk#13 AS item_sk#18, count(1)#17 AS cnt#19] -(18) Filter [codegen id : 4] +(20) Filter [codegen id : 2] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(19) Project [codegen id : 4] +(21) Project [codegen id : 2] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] -(20) BroadcastExchange +(22) BroadcastExchange Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(21) BroadcastHashJoin [codegen id : 5] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(22) Project [codegen id : 5] +(24) Project [codegen id : 3] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(23) Exchange +(25) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) Sort [codegen id : 6] +(26) Sort [codegen id : 4] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(25) Scan parquet spark_catalog.default.store_sales +(27) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(26) CometFilter +(28) CometFilter Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Condition : isnotnull(ss_customer_sk#20) -(27) CometProject +(29) CometProject Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -(28) ColumnarToRow [codegen id : 8] -Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(29) Scan parquet spark_catalog.default.customer +(30) Scan parquet spark_catalog.default.customer Output [1]: [c_customer_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(30) CometFilter +(31) CometFilter Input [1]: [c_customer_sk#24] Condition : isnotnull(c_customer_sk#24) -(31) ColumnarToRow [codegen id : 7] +(32) CometBroadcastExchange Input [1]: [c_customer_sk#24] +Arguments: [c_customer_sk#24] -(32) BroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(33) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] +Right output [1]: [c_customer_sk#24] +Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight -(33) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#20] -Right keys [1]: [c_customer_sk#24] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 8] -Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +(34) CometProject Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -(35) HashAggregate [codegen id : 8] +(35) CometHashAggregate Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Keys [1]: [c_customer_sk#24] Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [2]: [sum#25, isEmpty#26] -Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(36) Exchange -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(36) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] + +(37) Exchange +Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] +Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(37) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(38) HashAggregate [codegen id : 6] +Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28] -(38) Filter [codegen id : 9] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) +(39) Filter [codegen id : 6] +Input [2]: [c_customer_sk#24, ssales#28] +Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#29, [id=#30]))) -(39) Project [codegen id : 9] +(40) Project [codegen id : 6] Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] +Input [2]: [c_customer_sk#24, ssales#28] -(40) Sort [codegen id : 9] +(41) Sort [codegen id : 6] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(41) SortMergeJoin [codegen id : 11] +(42) SortMergeJoin [codegen id : 8] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(42) Project [codegen id : 11] +(43) Project [codegen id : 8] Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(43) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#33] +(44) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#31] -(44) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#33] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(45) Project [codegen id : 11] -Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#34] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#33] +(46) Project [codegen id : 8] +Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#31] -(46) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +(47) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#39), dynamicpruningexpression(ws_sold_date_sk#39 IN dynamicpruning#40)] +PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] -Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +(48) ColumnarToRow [codegen id : 11] +Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(48) ReusedExchange [Reuses operator id: 20] +(49) ReusedExchange [Reuses operator id: 22] Output [1]: [item_sk#18] -(49) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#35] +(50) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ws_item_sk#33] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(50) Project [codegen id : 16] -Output [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +(51) Project [codegen id : 11] +Output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(51) Exchange -Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -Arguments: hashpartitioning(ws_bill_customer_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(52) Exchange +Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(52) Sort [codegen id : 17] -Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -Arguments: [ws_bill_customer_sk#36 ASC NULLS FIRST], false, 0 +(53) Sort [codegen id : 12] +Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Arguments: [ws_bill_customer_sk#34 ASC NULLS FIRST], false, 0 -(53) ReusedExchange [Reuses operator id: 36] -Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(54) ReusedExchange [Reuses operator id: 37] +Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] -(54) HashAggregate [codegen id : 20] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(55) HashAggregate [codegen id : 14] +Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28] -(55) Filter [codegen id : 20] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) +(56) Filter [codegen id : 14] +Input [2]: [c_customer_sk#24, ssales#28] +Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30]))) -(56) Project [codegen id : 20] +(57) Project [codegen id : 14] Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] +Input [2]: [c_customer_sk#24, ssales#28] -(57) Sort [codegen id : 20] +(58) Sort [codegen id : 14] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(58) SortMergeJoin [codegen id : 22] -Left keys [1]: [ws_bill_customer_sk#36] +(59) SortMergeJoin [codegen id : 16] +Left keys [1]: [ws_bill_customer_sk#34] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(59) Project [codegen id : 22] -Output [3]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] -Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +(60) Project [codegen id : 16] +Output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] +Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -(60) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#41] +(61) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#39] -(61) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#39] -Right keys [1]: [d_date_sk#41] +(62) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#37] +Right keys [1]: [d_date_sk#39] Join type: Inner Join condition: None -(62) Project [codegen id : 22] -Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#42] -Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#41] +(63) Project [codegen id : 16] +Output [1]: [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#40] +Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#39] -(63) Union +(64) Union -(64) HashAggregate [codegen id : 23] -Input [1]: [sales#34] +(65) HashAggregate [codegen id : 17] +Input [1]: [sales#32] Keys: [] -Functions [1]: [partial_sum(sales#34)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [2]: [sum#45, isEmpty#46] +Functions [1]: [partial_sum(sales#32)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [2]: [sum#43, isEmpty#44] -(65) Exchange -Input [2]: [sum#45, isEmpty#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] +(66) Exchange +Input [2]: [sum#43, isEmpty#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(66) HashAggregate [codegen id : 24] -Input [2]: [sum#45, isEmpty#46] +(67) HashAggregate [codegen id : 18] +Input [2]: [sum#43, isEmpty#44] Keys: [] -Functions [1]: [sum(sales#34)] -Aggregate Attributes [1]: [sum(sales#34)#47] -Results [1]: [sum(sales#34)#47 AS sum(sales)#48] +Functions [1]: [sum(sales#32)] +Aggregate Attributes [1]: [sum(sales#32)#45] +Results [1]: [sum(sales#32)#45 AS sum(sales)#46] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (71) -+- * ColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan parquet spark_catalog.default.date_dim (67) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(67) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#33, d_year#49, d_moy#50] +(68) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#31, d_year#47, d_moy#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(68) CometFilter -Input [3]: [d_date_sk#33, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2000)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#33)) +(69) CometFilter +Input [3]: [d_date_sk#31, d_year#47, d_moy#48] +Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 2000)) AND (d_moy#48 = 2)) AND isnotnull(d_date_sk#31)) -(69) CometProject -Input [3]: [d_date_sk#33, d_year#49, d_moy#50] -Arguments: [d_date_sk#33], [d_date_sk#33] +(70) CometProject +Input [3]: [d_date_sk#31, d_year#47, d_moy#48] +Arguments: [d_date_sk#31], [d_date_sk#31] -(70) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] +(71) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#31] -(71) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +(72) BroadcastExchange +Input [1]: [d_date_sk#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * ColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan parquet spark_catalog.default.date_dim (72) +BroadcastExchange (77) ++- * ColumnarToRow (76) + +- CometProject (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(72) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#51] +(73) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_date#11, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#51] -Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +(74) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(74) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#51] +(75) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(75) ColumnarToRow [codegen id : 1] +(76) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(76) BroadcastExchange +(77) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (91) -+- Exchange (90) - +- * HashAggregate (89) - +- * HashAggregate (88) - +- Exchange (87) - +- * HashAggregate (86) - +- * Project (85) - +- * BroadcastHashJoin Inner BuildRight (84) - :- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * ColumnarToRow (79) - : : +- CometFilter (78) - : : +- CometScan parquet spark_catalog.default.store_sales (77) - : +- ReusedExchange (80) - +- ReusedExchange (83) - - -(77) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#29, [id=#30] +* HashAggregate (95) ++- Exchange (94) + +- * HashAggregate (93) + +- * HashAggregate (92) + +- Exchange (91) + +- * ColumnarToRow (90) + +- CometHashAggregate (89) + +- CometProject (88) + +- CometBroadcastHashJoin (87) + :- CometProject (82) + : +- CometBroadcastHashJoin (81) + : :- CometFilter (79) + : : +- CometScan parquet spark_catalog.default.store_sales (78) + : +- ReusedExchange (80) + +- CometBroadcastExchange (86) + +- CometProject (85) + +- CometFilter (84) + +- CometScan parquet spark_catalog.default.date_dim (83) + + +(78) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#55), dynamicpruningexpression(ss_sold_date_sk#55 IN dynamicpruning#56)] +PartitionFilters: [isnotnull(ss_sold_date_sk#52), dynamicpruningexpression(ss_sold_date_sk#52 IN dynamicpruning#53)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(78) CometFilter -Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] -Condition : isnotnull(ss_customer_sk#52) - -(79) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] +(79) CometFilter +Input [4]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52] +Condition : isnotnull(ss_customer_sk#49) (80) ReusedExchange [Reuses operator id: 32] -Output [1]: [c_customer_sk#57] +Output [1]: [c_customer_sk#54] -(81) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#52] -Right keys [1]: [c_customer_sk#57] -Join type: Inner -Join condition: None +(81) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52] +Right output [1]: [c_customer_sk#54] +Arguments: [ss_customer_sk#49], [c_customer_sk#54], Inner, BuildRight -(82) Project [codegen id : 3] -Output [4]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] -Input [5]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] +(82) CometProject +Input [5]: [ss_customer_sk#49, ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54] +Arguments: [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54], [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54] -(83) ReusedExchange [Reuses operator id: 96] -Output [1]: [d_date_sk#58] +(83) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#55, d_year#56] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct -(84) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#55] -Right keys [1]: [d_date_sk#58] -Join type: Inner -Join condition: None +(84) CometFilter +Input [2]: [d_date_sk#55, d_year#56] +Condition : (d_year#56 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#55)) + +(85) CometProject +Input [2]: [d_date_sk#55, d_year#56] +Arguments: [d_date_sk#55], [d_date_sk#55] + +(86) CometBroadcastExchange +Input [1]: [d_date_sk#55] +Arguments: [d_date_sk#55] + +(87) CometBroadcastHashJoin +Left output [4]: [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54] +Right output [1]: [d_date_sk#55] +Arguments: [ss_sold_date_sk#52], [d_date_sk#55], Inner, BuildRight + +(88) CometProject +Input [5]: [ss_quantity#50, ss_sales_price#51, ss_sold_date_sk#52, c_customer_sk#54, d_date_sk#55] +Arguments: [ss_quantity#50, ss_sales_price#51, c_customer_sk#54], [ss_quantity#50, ss_sales_price#51, c_customer_sk#54] + +(89) CometHashAggregate +Input [3]: [ss_quantity#50, ss_sales_price#51, c_customer_sk#54] +Keys [1]: [c_customer_sk#54] +Functions [1]: [partial_sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))] + +(90) ColumnarToRow [codegen id : 1] +Input [3]: [c_customer_sk#54, sum#57, isEmpty#58] + +(91) Exchange +Input [3]: [c_customer_sk#54, sum#57, isEmpty#58] +Arguments: hashpartitioning(c_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(92) HashAggregate [codegen id : 2] +Input [3]: [c_customer_sk#54, sum#57, isEmpty#58] +Keys [1]: [c_customer_sk#54] +Functions [1]: [sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))#59] +Results [1]: [sum((cast(ss_quantity#50 as decimal(10,0)) * ss_sales_price#51))#59 AS csales#60] -(85) Project [codegen id : 3] -Output [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] -Input [5]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57, d_date_sk#58] - -(86) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] -Keys [1]: [c_customer_sk#57] -Functions [1]: [partial_sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] -Aggregate Attributes [2]: [sum#59, isEmpty#60] -Results [3]: [c_customer_sk#57, sum#61, isEmpty#62] - -(87) Exchange -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -Arguments: hashpartitioning(c_customer_sk#57, 5), ENSURE_REQUIREMENTS, [plan_id=11] - -(88) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -Keys [1]: [c_customer_sk#57] -Functions [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63] -Results [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63 AS csales#64] - -(89) HashAggregate [codegen id : 4] -Input [1]: [csales#64] +(93) HashAggregate [codegen id : 2] +Input [1]: [csales#60] Keys: [] -Functions [1]: [partial_max(csales#64)] -Aggregate Attributes [1]: [max#65] -Results [1]: [max#66] +Functions [1]: [partial_max(csales#60)] +Aggregate Attributes [1]: [max#61] +Results [1]: [max#62] -(90) Exchange -Input [1]: [max#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(94) Exchange +Input [1]: [max#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(91) HashAggregate [codegen id : 5] -Input [1]: [max#66] +(95) HashAggregate [codegen id : 3] +Input [1]: [max#62] Keys: [] -Functions [1]: [max(csales#64)] -Aggregate Attributes [1]: [max(csales#64)#67] -Results [1]: [max(csales#64)#67 AS tpcds_cmax#68] +Functions [1]: [max(csales#60)] +Aggregate Attributes [1]: [max(csales#60)#63] +Results [1]: [max(csales#60)#63 AS tpcds_cmax#64] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#55 IN dynamicpruning#56 -BroadcastExchange (96) -+- * ColumnarToRow (95) - +- CometProject (94) - +- CometFilter (93) - +- CometScan parquet spark_catalog.default.date_dim (92) +Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#52 IN dynamicpruning#53 +BroadcastExchange (100) ++- * ColumnarToRow (99) + +- CometProject (98) + +- CometFilter (97) + +- CometScan parquet spark_catalog.default.date_dim (96) -(92) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#58, d_year#69] +(96) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#55, d_year#56] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(93) CometFilter -Input [2]: [d_date_sk#58, d_year#69] -Condition : (d_year#69 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#58)) +(97) CometFilter +Input [2]: [d_date_sk#55, d_year#56] +Condition : (d_year#56 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#55)) -(94) CometProject -Input [2]: [d_date_sk#58, d_year#69] -Arguments: [d_date_sk#58], [d_date_sk#58] +(98) CometProject +Input [2]: [d_date_sk#55, d_year#56] +Arguments: [d_date_sk#55], [d_date_sk#55] -(95) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#58] +(99) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#55] -(96) BroadcastExchange -Input [1]: [d_date_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(100) BroadcastExchange +Input [1]: [d_date_sk#55] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:5 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:6 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt index 0ec56d0e72..f818fd25f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt @@ -1,22 +1,22 @@ -WholeStageCodegen (24) +WholeStageCodegen (18) HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] InputAdapter Exchange #1 - WholeStageCodegen (23) + WholeStageCodegen (17) HashAggregate [sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (11) + WholeStageCodegen (8) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_quantity,cs_list_price,cs_sold_date_sk] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (4) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) + WholeStageCodegen (3) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow @@ -32,20 +32,20 @@ WholeStageCodegen (24) CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [item_sk] Filter [cnt] HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] InputAdapter Exchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 @@ -56,84 +56,79 @@ WholeStageCodegen (24) CometProject [d_date_sk,d_date] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange #7 + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange #8 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - WholeStageCodegen (9) + WholeStageCodegen (6) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] Subquery #3 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [max] [max(csales),tpcds_cmax,max] InputAdapter - Exchange #10 - WholeStageCodegen (4) + Exchange #11 + WholeStageCodegen (2) HashAggregate [csales] [max,max] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] InputAdapter - Exchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_sk] #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 + BroadcastExchange #13 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [c_customer_sk] #9 - InputAdapter - ReusedExchange [d_date_sk] #12 + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange #14 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter - Exchange [c_customer_sk] #8 - WholeStageCodegen (8) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_sk] #9 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange #10 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) + WholeStageCodegen (16) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_quantity,ws_list_price,ws_sold_date_sk] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (17) + WholeStageCodegen (12) Sort [ws_bill_customer_sk] InputAdapter - Exchange [ws_bill_customer_sk] #13 - WholeStageCodegen (16) + Exchange [ws_bill_customer_sk] #15 + WholeStageCodegen (11) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow @@ -143,13 +138,13 @@ WholeStageCodegen (24) InputAdapter ReusedExchange [item_sk] #4 InputAdapter - WholeStageCodegen (20) + WholeStageCodegen (14) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] ReusedSubquery [tpcds_cmax] #3 HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [c_customer_sk,sum,isEmpty] #9 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index c36e18596d..beb9757538 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -1,91 +1,92 @@ == Physical Plan == -TakeOrderedAndProject (87) -+- Union (86) - :- * HashAggregate (62) - : +- Exchange (61) - : +- * HashAggregate (60) - : +- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (56) - : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * SortMergeJoin LeftSemi (42) - : : : :- * Sort (25) - : : : : +- Exchange (24) - : : : : +- * Project (23) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (22) +TakeOrderedAndProject (88) ++- Union (87) + :- * HashAggregate (63) + : +- Exchange (62) + : +- * HashAggregate (61) + : +- * Project (60) + : +- * BroadcastHashJoin Inner BuildRight (59) + : :- * Project (57) + : : +- * BroadcastHashJoin Inner BuildRight (56) + : : :- * SortMergeJoin LeftSemi (43) + : : : :- * Sort (27) + : : : : +- Exchange (26) + : : : : +- * Project (25) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (24) : : : : :- * ColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (21) - : : : : +- * Project (20) - : : : : +- * Filter (19) - : : : : +- * HashAggregate (18) - : : : : +- Exchange (17) - : : : : +- * HashAggregate (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (7) - : : : : +- BroadcastExchange (13) - : : : : +- * ColumnarToRow (12) - : : : : +- CometFilter (11) - : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : +- * Sort (41) - : : : +- * Project (40) - : : : +- * Filter (39) - : : : +- * HashAggregate (38) - : : : +- Exchange (37) - : : : +- * HashAggregate (36) - : : : +- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (29) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan parquet spark_catalog.default.store_sales (26) - : : : +- BroadcastExchange (33) - : : : +- * ColumnarToRow (32) - : : : +- CometFilter (31) - : : : +- CometScan parquet spark_catalog.default.customer (30) - : : +- BroadcastExchange (54) - : : +- * SortMergeJoin LeftSemi (53) - : : :- * Sort (47) - : : : +- Exchange (46) - : : : +- * ColumnarToRow (45) - : : : +- CometFilter (44) - : : : +- CometScan parquet spark_catalog.default.customer (43) - : : +- * Sort (52) - : : +- * Project (51) - : : +- * Filter (50) - : : +- * HashAggregate (49) - : : +- ReusedExchange (48) - : +- ReusedExchange (57) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * BroadcastHashJoin Inner BuildRight (81) - :- * Project (79) - : +- * BroadcastHashJoin Inner BuildRight (78) - : :- * SortMergeJoin LeftSemi (76) - : : :- * Sort (70) - : : : +- Exchange (69) - : : : +- * Project (68) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (67) - : : : :- * ColumnarToRow (65) - : : : : +- CometFilter (64) - : : : : +- CometScan parquet spark_catalog.default.web_sales (63) - : : : +- ReusedExchange (66) - : : +- * Sort (75) - : : +- * Project (74) - : : +- * Filter (73) - : : +- * HashAggregate (72) - : : +- ReusedExchange (71) - : +- ReusedExchange (77) - +- ReusedExchange (80) + : : : : +- BroadcastExchange (23) + : : : : +- * Project (22) + : : : : +- * Filter (21) + : : : : +- * HashAggregate (20) + : : : : +- Exchange (19) + : : : : +- * ColumnarToRow (18) + : : : : +- CometHashAggregate (17) + : : : : +- CometProject (16) + : : : : +- CometBroadcastHashJoin (15) + : : : : :- CometProject (11) + : : : : : +- CometBroadcastHashJoin (10) + : : : : : :- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : : +- CometBroadcastExchange (9) + : : : : : +- CometProject (8) + : : : : : +- CometFilter (7) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (6) + : : : : +- CometBroadcastExchange (14) + : : : : +- CometFilter (13) + : : : : +- CometScan parquet spark_catalog.default.item (12) + : : : +- * Sort (42) + : : : +- * Project (41) + : : : +- * Filter (40) + : : : +- * HashAggregate (39) + : : : +- Exchange (38) + : : : +- * ColumnarToRow (37) + : : : +- CometHashAggregate (36) + : : : +- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometProject (30) + : : : : +- CometFilter (29) + : : : : +- CometScan parquet spark_catalog.default.store_sales (28) + : : : +- CometBroadcastExchange (33) + : : : +- CometFilter (32) + : : : +- CometScan parquet spark_catalog.default.customer (31) + : : +- BroadcastExchange (55) + : : +- * SortMergeJoin LeftSemi (54) + : : :- * Sort (48) + : : : +- Exchange (47) + : : : +- * ColumnarToRow (46) + : : : +- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.customer (44) + : : +- * Sort (53) + : : +- * Project (52) + : : +- * Filter (51) + : : +- * HashAggregate (50) + : : +- ReusedExchange (49) + : +- ReusedExchange (58) + +- * HashAggregate (86) + +- Exchange (85) + +- * HashAggregate (84) + +- * Project (83) + +- * BroadcastHashJoin Inner BuildRight (82) + :- * Project (80) + : +- * BroadcastHashJoin Inner BuildRight (79) + : :- * SortMergeJoin LeftSemi (77) + : : :- * Sort (71) + : : : +- Exchange (70) + : : : +- * Project (69) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (68) + : : : :- * ColumnarToRow (66) + : : : : +- CometFilter (65) + : : : : +- CometScan parquet spark_catalog.default.web_sales (64) + : : : +- ReusedExchange (67) + : : +- * Sort (76) + : : +- * Project (75) + : : +- * Filter (74) + : : +- * HashAggregate (73) + : : +- ReusedExchange (72) + : +- ReusedExchange (78) + +- ReusedExchange (81) (1) Scan parquet spark_catalog.default.catalog_sales @@ -100,7 +101,7 @@ ReadSchema: struct Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) -(6) ColumnarToRow [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] +(6) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_date#11, d_year#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct -(7) ReusedExchange [Reuses operator id: 97] -Output [2]: [d_date_sk#10, d_date#11] +(7) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(8) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] + +(9) CometBroadcastExchange +Input [2]: [d_date_sk#10, d_date#11] +Arguments: [d_date_sk#10, d_date#11] -(9) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#11] +(10) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] +Right output [2]: [d_date_sk#10, d_date#11] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight + +(11) CometProject Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] +Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] -(10) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#12, i_item_desc#13] +(12) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#13, i_item_desc#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(11) CometFilter -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) +(13) CometFilter +Input [2]: [i_item_sk#13, i_item_desc#14] +Condition : isnotnull(i_item_sk#13) -(12) ColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] +(14) CometBroadcastExchange +Input [2]: [i_item_sk#13, i_item_desc#14] +Arguments: [i_item_sk#13, i_item_desc#14] -(13) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(15) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#7, d_date#11] +Right output [2]: [i_item_sk#13, i_item_desc#14] +Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join type: Inner -Join condition: None +(16) CometProject +Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] +Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] -(15) Project [codegen id : 3] -Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] - -(16) HashAggregate [codegen id : 3] -Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +(17) CometHashAggregate +Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] +Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(17) Exchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(18) ColumnarToRow [codegen id : 1] +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] + +(19) Exchange +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] +Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(18) HashAggregate [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +(20) HashAggregate [codegen id : 2] +Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] +Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] +Results [2]: [i_item_sk#13 AS item_sk#18, count(1)#17 AS cnt#19] -(19) Filter [codegen id : 4] +(21) Filter [codegen id : 2] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(20) Project [codegen id : 4] +(22) Project [codegen id : 2] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] -(21) BroadcastExchange +(23) BroadcastExchange Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(22) BroadcastHashJoin [codegen id : 5] +(24) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(23) Project [codegen id : 5] +(25) Project [codegen id : 3] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(24) Exchange +(26) Exchange Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(25) Sort [codegen id : 6] +(27) Sort [codegen id : 4] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 -(26) Scan parquet spark_catalog.default.store_sales +(28) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(27) CometFilter +(29) CometFilter Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Condition : isnotnull(ss_customer_sk#20) -(28) CometProject +(30) CometProject Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -(29) ColumnarToRow [codegen id : 8] -Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(30) Scan parquet spark_catalog.default.customer +(31) Scan parquet spark_catalog.default.customer Output [1]: [c_customer_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [1]: [c_customer_sk#24] Condition : isnotnull(c_customer_sk#24) -(32) ColumnarToRow [codegen id : 7] +(33) CometBroadcastExchange Input [1]: [c_customer_sk#24] +Arguments: [c_customer_sk#24] -(33) BroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(34) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] +Right output [1]: [c_customer_sk#24] +Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#20] -Right keys [1]: [c_customer_sk#24] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 8] -Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +(35) CometProject Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -(36) HashAggregate [codegen id : 8] +(36) CometHashAggregate Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] Keys [1]: [c_customer_sk#24] Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [2]: [sum#25, isEmpty#26] -Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] -(37) Exchange -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(37) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] + +(38) Exchange +Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] +Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(38) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(39) HashAggregate [codegen id : 6] +Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28] -(39) Filter [codegen id : 9] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) +(40) Filter [codegen id : 6] +Input [2]: [c_customer_sk#24, ssales#28] +Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#29, [id=#30]))) -(40) Project [codegen id : 9] +(41) Project [codegen id : 6] Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] +Input [2]: [c_customer_sk#24, ssales#28] -(41) Sort [codegen id : 9] +(42) Sort [codegen id : 6] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(42) SortMergeJoin [codegen id : 17] +(43) SortMergeJoin [codegen id : 13] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(43) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +(44) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(44) CometFilter -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Condition : isnotnull(c_customer_sk#33) +(45) CometFilter +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Condition : isnotnull(c_customer_sk#31) -(45) ColumnarToRow [codegen id : 10] -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +(46) ColumnarToRow [codegen id : 7] +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] -(46) Exchange -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: hashpartitioning(c_customer_sk#33, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(47) Exchange +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(47) Sort [codegen id : 11] -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#33 ASC NULLS FIRST], false, 0 +(48) Sort [codegen id : 8] +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 -(48) ReusedExchange [Reuses operator id: 37] -Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(49) ReusedExchange [Reuses operator id: 38] +Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] -(49) HashAggregate [codegen id : 14] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(50) HashAggregate [codegen id : 10] +Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28] -(50) Filter [codegen id : 14] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) +(51) Filter [codegen id : 10] +Input [2]: [c_customer_sk#24, ssales#28] +Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30]))) -(51) Project [codegen id : 14] +(52) Project [codegen id : 10] Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] +Input [2]: [c_customer_sk#24, ssales#28] -(52) Sort [codegen id : 14] +(53) Sort [codegen id : 10] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(53) SortMergeJoin [codegen id : 15] -Left keys [1]: [c_customer_sk#33] +(54) SortMergeJoin [codegen id : 11] +Left keys [1]: [c_customer_sk#31] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(54) BroadcastExchange -Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(55) BroadcastExchange +Input [3]: [c_customer_sk#31, c_first_name#32, c_last_name#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(55) BroadcastHashJoin [codegen id : 17] +(56) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#33] +Right keys [1]: [c_customer_sk#31] Join type: Inner Join condition: None -(56) Project [codegen id : 17] -Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35] -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#33, c_first_name#34, c_last_name#35] +(57) Project [codegen id : 13] +Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#31, c_first_name#32, c_last_name#33] -(57) ReusedExchange [Reuses operator id: 92] -Output [1]: [d_date_sk#36] +(58) ReusedExchange [Reuses operator id: 93] +Output [1]: [d_date_sk#34] -(58) BroadcastHashJoin [codegen id : 17] +(59) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#36] +Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(59) Project [codegen id : 17] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35, d_date_sk#36] +(60) Project [codegen id : 13] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33, d_date_sk#34] -(60) HashAggregate [codegen id : 17] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] -Keys [2]: [c_last_name#35, c_first_name#34] +(61) HashAggregate [codegen id : 13] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] +Keys [2]: [c_last_name#33, c_first_name#32] Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] -Aggregate Attributes [2]: [sum#37, isEmpty#38] -Results [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] +Aggregate Attributes [2]: [sum#35, isEmpty#36] +Results [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -(61) Exchange -Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(62) Exchange +Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] +Arguments: hashpartitioning(c_last_name#33, c_first_name#32, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(62) HashAggregate [codegen id : 18] -Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] -Keys [2]: [c_last_name#35, c_first_name#34] +(63) HashAggregate [codegen id : 14] +Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] +Keys [2]: [c_last_name#33, c_first_name#32] Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] -Aggregate Attributes [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41] -Results [3]: [c_last_name#35, c_first_name#34, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41 AS sales#42] +Aggregate Attributes [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#39] +Results [3]: [c_last_name#33, c_first_name#32, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#39 AS sales#40] -(63) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +(64) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] +PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#46)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) CometFilter -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_customer_sk#44) +(65) CometFilter +Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +Condition : isnotnull(ws_bill_customer_sk#42) -(65) ColumnarToRow [codegen id : 23] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +(66) ColumnarToRow [codegen id : 17] +Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -(66) ReusedExchange [Reuses operator id: 21] +(67) ReusedExchange [Reuses operator id: 23] Output [1]: [item_sk#18] -(67) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ws_item_sk#43] +(68) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#41] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(68) Project [codegen id : 23] -Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +(69) Project [codegen id : 17] +Output [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +Input [5]: [ws_item_sk#41, ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] -(69) Exchange -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(70) Exchange +Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(70) Sort [codegen id : 24] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 +(71) Sort [codegen id : 18] +Input [4]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45] +Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 -(71) ReusedExchange [Reuses operator id: 37] -Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(72) ReusedExchange [Reuses operator id: 38] +Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] -(72) HashAggregate [codegen id : 27] -Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +(73) HashAggregate [codegen id : 20] +Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] Keys [1]: [c_customer_sk#24] Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] -Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#27 AS ssales#28] -(73) Filter [codegen id : 27] -Input [2]: [c_customer_sk#24, ssales#30] -Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) +(74) Filter [codegen id : 20] +Input [2]: [c_customer_sk#24, ssales#28] +Condition : (isnotnull(ssales#28) AND (cast(ssales#28 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#29, [id=#30]))) -(74) Project [codegen id : 27] +(75) Project [codegen id : 20] Output [1]: [c_customer_sk#24] -Input [2]: [c_customer_sk#24, ssales#30] +Input [2]: [c_customer_sk#24, ssales#28] -(75) Sort [codegen id : 27] +(76) Sort [codegen id : 20] Input [1]: [c_customer_sk#24] Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 -(76) SortMergeJoin [codegen id : 35] -Left keys [1]: [ws_bill_customer_sk#44] +(77) SortMergeJoin [codegen id : 27] +Left keys [1]: [ws_bill_customer_sk#42] Right keys [1]: [c_customer_sk#24] Join type: LeftSemi Join condition: None -(77) ReusedExchange [Reuses operator id: 54] -Output [3]: [c_customer_sk#49, c_first_name#50, c_last_name#51] +(78) ReusedExchange [Reuses operator id: 55] +Output [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] -(78) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#49] +(79) BroadcastHashJoin [codegen id : 27] +Left keys [1]: [ws_bill_customer_sk#42] +Right keys [1]: [c_customer_sk#47] Join type: Inner Join condition: None -(79) Project [codegen id : 35] -Output [5]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#50, c_last_name#51] -Input [7]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_customer_sk#49, c_first_name#50, c_last_name#51] +(80) Project [codegen id : 27] +Output [5]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, c_first_name#48, c_last_name#49] +Input [7]: [ws_bill_customer_sk#42, ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, c_customer_sk#47, c_first_name#48, c_last_name#49] -(80) ReusedExchange [Reuses operator id: 92] -Output [1]: [d_date_sk#52] +(81) ReusedExchange [Reuses operator id: 93] +Output [1]: [d_date_sk#50] -(81) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#52] +(82) BroadcastHashJoin [codegen id : 27] +Left keys [1]: [ws_sold_date_sk#45] +Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(82) Project [codegen id : 35] -Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] -Input [6]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#50, c_last_name#51, d_date_sk#52] +(83) Project [codegen id : 27] +Output [4]: [ws_quantity#43, ws_list_price#44, c_first_name#48, c_last_name#49] +Input [6]: [ws_quantity#43, ws_list_price#44, ws_sold_date_sk#45, c_first_name#48, c_last_name#49, d_date_sk#50] -(83) HashAggregate [codegen id : 35] -Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51] -Keys [2]: [c_last_name#51, c_first_name#50] -Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] -Aggregate Attributes [2]: [sum#53, isEmpty#54] -Results [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] +(84) HashAggregate [codegen id : 27] +Input [4]: [ws_quantity#43, ws_list_price#44, c_first_name#48, c_last_name#49] +Keys [2]: [c_last_name#49, c_first_name#48] +Functions [1]: [partial_sum((cast(ws_quantity#43 as decimal(10,0)) * ws_list_price#44))] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] -(84) Exchange -Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] -Arguments: hashpartitioning(c_last_name#51, c_first_name#50, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(85) Exchange +Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] +Arguments: hashpartitioning(c_last_name#49, c_first_name#48, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(85) HashAggregate [codegen id : 36] -Input [4]: [c_last_name#51, c_first_name#50, sum#55, isEmpty#56] -Keys [2]: [c_last_name#51, c_first_name#50] -Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] -Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#57] -Results [3]: [c_last_name#51, c_first_name#50, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#57 AS sales#58] +(86) HashAggregate [codegen id : 28] +Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] +Keys [2]: [c_last_name#49, c_first_name#48] +Functions [1]: [sum((cast(ws_quantity#43 as decimal(10,0)) * ws_list_price#44))] +Aggregate Attributes [1]: [sum((cast(ws_quantity#43 as decimal(10,0)) * ws_list_price#44))#55] +Results [3]: [c_last_name#49, c_first_name#48, sum((cast(ws_quantity#43 as decimal(10,0)) * ws_list_price#44))#55 AS sales#56] -(86) Union +(87) Union -(87) TakeOrderedAndProject -Input [3]: [c_last_name#35, c_first_name#34, sales#42] -Arguments: 100, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, sales#42 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, sales#42] +(88) TakeOrderedAndProject +Input [3]: [c_last_name#33, c_first_name#32, sales#40] +Arguments: 100, [c_last_name#33 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, sales#40 ASC NULLS FIRST], [c_last_name#33, c_first_name#32, sales#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (92) -+- * ColumnarToRow (91) - +- CometProject (90) - +- CometFilter (89) - +- CometScan parquet spark_catalog.default.date_dim (88) +BroadcastExchange (93) ++- * ColumnarToRow (92) + +- CometProject (91) + +- CometFilter (90) + +- CometScan parquet spark_catalog.default.date_dim (89) -(88) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#36, d_year#59, d_moy#60] +(89) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#34, d_year#57, d_moy#58] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(89) CometFilter -Input [3]: [d_date_sk#36, d_year#59, d_moy#60] -Condition : ((((isnotnull(d_year#59) AND isnotnull(d_moy#60)) AND (d_year#59 = 2000)) AND (d_moy#60 = 2)) AND isnotnull(d_date_sk#36)) +(90) CometFilter +Input [3]: [d_date_sk#34, d_year#57, d_moy#58] +Condition : ((((isnotnull(d_year#57) AND isnotnull(d_moy#58)) AND (d_year#57 = 2000)) AND (d_moy#58 = 2)) AND isnotnull(d_date_sk#34)) -(90) CometProject -Input [3]: [d_date_sk#36, d_year#59, d_moy#60] -Arguments: [d_date_sk#36], [d_date_sk#36] +(91) CometProject +Input [3]: [d_date_sk#34, d_year#57, d_moy#58] +Arguments: [d_date_sk#34], [d_date_sk#34] -(91) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#36] +(92) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#34] -(92) BroadcastExchange -Input [1]: [d_date_sk#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +(93) BroadcastExchange +Input [1]: [d_date_sk#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (97) -+- * ColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometScan parquet spark_catalog.default.date_dim (93) +BroadcastExchange (98) ++- * ColumnarToRow (97) + +- CometProject (96) + +- CometFilter (95) + +- CometScan parquet spark_catalog.default.date_dim (94) -(93) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#61] +(94) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_date#11, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(94) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#61] -Condition : (d_year#61 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +(95) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(95) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#61] +(96) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(96) ColumnarToRow [codegen id : 1] +(97) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(97) BroadcastExchange +(98) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#31, [id=#32] -* HashAggregate (112) -+- Exchange (111) - +- * HashAggregate (110) - +- * HashAggregate (109) - +- Exchange (108) - +- * HashAggregate (107) - +- * Project (106) - +- * BroadcastHashJoin Inner BuildRight (105) - :- * Project (103) - : +- * BroadcastHashJoin Inner BuildRight (102) - : :- * ColumnarToRow (100) - : : +- CometFilter (99) - : : +- CometScan parquet spark_catalog.default.store_sales (98) - : +- ReusedExchange (101) - +- ReusedExchange (104) - - -(98) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] + +Subquery:3 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#29, [id=#30] +* HashAggregate (116) ++- Exchange (115) + +- * HashAggregate (114) + +- * HashAggregate (113) + +- Exchange (112) + +- * ColumnarToRow (111) + +- CometHashAggregate (110) + +- CometProject (109) + +- CometBroadcastHashJoin (108) + :- CometProject (103) + : +- CometBroadcastHashJoin (102) + : :- CometFilter (100) + : : +- CometScan parquet spark_catalog.default.store_sales (99) + : +- ReusedExchange (101) + +- CometBroadcastExchange (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) + + +(99) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#65), dynamicpruningexpression(ss_sold_date_sk#65 IN dynamicpruning#66)] +PartitionFilters: [isnotnull(ss_sold_date_sk#62), dynamicpruningexpression(ss_sold_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(99) CometFilter -Input [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] -Condition : isnotnull(ss_customer_sk#62) - -(100) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65] +(100) CometFilter +Input [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] +Condition : isnotnull(ss_customer_sk#59) (101) ReusedExchange [Reuses operator id: 33] -Output [1]: [c_customer_sk#67] +Output [1]: [c_customer_sk#64] -(102) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#62] -Right keys [1]: [c_customer_sk#67] -Join type: Inner -Join condition: None +(102) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] +Right output [1]: [c_customer_sk#64] +Arguments: [ss_customer_sk#59], [c_customer_sk#64], Inner, BuildRight -(103) Project [codegen id : 3] -Output [4]: [ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67] -Input [5]: [ss_customer_sk#62, ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67] +(103) CometProject +Input [5]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64] +Arguments: [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64], [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64] -(104) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#68] +(104) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#65, d_year#66] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct -(105) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#65] -Right keys [1]: [d_date_sk#68] -Join type: Inner -Join condition: None +(105) CometFilter +Input [2]: [d_date_sk#65, d_year#66] +Condition : (d_year#66 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#65)) + +(106) CometProject +Input [2]: [d_date_sk#65, d_year#66] +Arguments: [d_date_sk#65], [d_date_sk#65] + +(107) CometBroadcastExchange +Input [1]: [d_date_sk#65] +Arguments: [d_date_sk#65] + +(108) CometBroadcastHashJoin +Left output [4]: [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64] +Right output [1]: [d_date_sk#65] +Arguments: [ss_sold_date_sk#62], [d_date_sk#65], Inner, BuildRight + +(109) CometProject +Input [5]: [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64, d_date_sk#65] +Arguments: [ss_quantity#60, ss_sales_price#61, c_customer_sk#64], [ss_quantity#60, ss_sales_price#61, c_customer_sk#64] + +(110) CometHashAggregate +Input [3]: [ss_quantity#60, ss_sales_price#61, c_customer_sk#64] +Keys [1]: [c_customer_sk#64] +Functions [1]: [partial_sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))] + +(111) ColumnarToRow [codegen id : 1] +Input [3]: [c_customer_sk#64, sum#67, isEmpty#68] + +(112) Exchange +Input [3]: [c_customer_sk#64, sum#67, isEmpty#68] +Arguments: hashpartitioning(c_customer_sk#64, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(113) HashAggregate [codegen id : 2] +Input [3]: [c_customer_sk#64, sum#67, isEmpty#68] +Keys [1]: [c_customer_sk#64] +Functions [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))#69] +Results [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))#69 AS csales#70] -(106) Project [codegen id : 3] -Output [3]: [ss_quantity#63, ss_sales_price#64, c_customer_sk#67] -Input [5]: [ss_quantity#63, ss_sales_price#64, ss_sold_date_sk#65, c_customer_sk#67, d_date_sk#68] - -(107) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#63, ss_sales_price#64, c_customer_sk#67] -Keys [1]: [c_customer_sk#67] -Functions [1]: [partial_sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))] -Aggregate Attributes [2]: [sum#69, isEmpty#70] -Results [3]: [c_customer_sk#67, sum#71, isEmpty#72] - -(108) Exchange -Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] -Arguments: hashpartitioning(c_customer_sk#67, 5), ENSURE_REQUIREMENTS, [plan_id=14] - -(109) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#67, sum#71, isEmpty#72] -Keys [1]: [c_customer_sk#67] -Functions [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))#73] -Results [1]: [sum((cast(ss_quantity#63 as decimal(10,0)) * ss_sales_price#64))#73 AS csales#74] - -(110) HashAggregate [codegen id : 4] -Input [1]: [csales#74] +(114) HashAggregate [codegen id : 2] +Input [1]: [csales#70] Keys: [] -Functions [1]: [partial_max(csales#74)] -Aggregate Attributes [1]: [max#75] -Results [1]: [max#76] +Functions [1]: [partial_max(csales#70)] +Aggregate Attributes [1]: [max#71] +Results [1]: [max#72] -(111) Exchange -Input [1]: [max#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] +(115) Exchange +Input [1]: [max#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(112) HashAggregate [codegen id : 5] -Input [1]: [max#76] +(116) HashAggregate [codegen id : 3] +Input [1]: [max#72] Keys: [] -Functions [1]: [max(csales#74)] -Aggregate Attributes [1]: [max(csales#74)#77] -Results [1]: [max(csales#74)#77 AS tpcds_cmax#78] +Functions [1]: [max(csales#70)] +Aggregate Attributes [1]: [max(csales#70)#73] +Results [1]: [max(csales#70)#73 AS tpcds_cmax#74] -Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#65 IN dynamicpruning#66 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#62 IN dynamicpruning#63 +BroadcastExchange (121) ++- * ColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan parquet spark_catalog.default.date_dim (117) -(113) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#68, d_year#79] +(117) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#65, d_year#66] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter -Input [2]: [d_date_sk#68, d_year#79] -Condition : (d_year#79 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#68)) +(118) CometFilter +Input [2]: [d_date_sk#65, d_year#66] +Condition : (d_year#66 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#65)) -(115) CometProject -Input [2]: [d_date_sk#68, d_year#79] -Arguments: [d_date_sk#68], [d_date_sk#68] +(119) CometProject +Input [2]: [d_date_sk#65, d_year#66] +Arguments: [d_date_sk#65], [d_date_sk#65] -(116) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#68] +(120) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#65] -(117) BroadcastExchange -Input [1]: [d_date_sk#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] +(121) BroadcastExchange +Input [1]: [d_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:5 Hosting operator id = 50 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:5 Hosting operator id = 51 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] -Subquery:6 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] +Subquery:7 Hosting operator id = 74 Hosting Expression = ReusedSubquery Subquery scalar-subquery#29, [id=#30] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 49ddeaef82..54ee3dbde7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union - WholeStageCodegen (18) + WholeStageCodegen (14) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty] InputAdapter Exchange [c_last_name,c_first_name] #1 - WholeStageCodegen (17) + WholeStageCodegen (13) HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_list_price,c_first_name,c_last_name] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -12,11 +12,11 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] SortMergeJoin [cs_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (4) Sort [cs_bill_customer_sk] InputAdapter Exchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) + WholeStageCodegen (3) Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] BroadcastHashJoin [cs_item_sk,item_sk] ColumnarToRow @@ -33,20 +33,20 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [item_sk] Filter [cnt] HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] InputAdapter Exchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [_groupingexpression,i_item_sk,d_date] + CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 @@ -57,101 +57,96 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] CometProject [d_date_sk,d_date] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange #7 + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometBroadcastExchange #8 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - WholeStageCodegen (9) + WholeStageCodegen (6) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] Subquery #3 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [max] [max(csales),tpcds_cmax,max] InputAdapter - Exchange #10 - WholeStageCodegen (4) + Exchange #11 + WholeStageCodegen (2) HashAggregate [csales] [max,max] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] InputAdapter - Exchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_sk] #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 + BroadcastExchange #13 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [c_customer_sk] #9 - InputAdapter - ReusedExchange [d_date_sk] #12 + ReusedExchange [c_customer_sk] #10 + CometBroadcastExchange #14 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter - Exchange [c_customer_sk] #8 - WholeStageCodegen (8) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_sk] #9 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_sk,ss_quantity,ss_sales_price] + CometProject [ss_quantity,ss_sales_price,c_customer_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] CometProject [ss_customer_sk,ss_quantity,ss_sales_price] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange #10 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (15) + BroadcastExchange #15 + WholeStageCodegen (11) SortMergeJoin [c_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (8) Sort [c_customer_sk] InputAdapter - Exchange [c_customer_sk] #14 - WholeStageCodegen (10) + Exchange [c_customer_sk] #16 + WholeStageCodegen (7) ColumnarToRow InputAdapter CometFilter [c_customer_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - WholeStageCodegen (14) + WholeStageCodegen (10) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] ReusedSubquery [tpcds_cmax] #3 HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [c_customer_sk,sum,isEmpty] #9 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (36) + WholeStageCodegen (28) HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty] InputAdapter - Exchange [c_last_name,c_first_name] #15 - WholeStageCodegen (35) + Exchange [c_last_name,c_first_name] #17 + WholeStageCodegen (27) HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_list_price,c_first_name,c_last_name] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -159,11 +154,11 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] SortMergeJoin [ws_bill_customer_sk,c_customer_sk] InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (18) Sort [ws_bill_customer_sk] InputAdapter - Exchange [ws_bill_customer_sk] #16 - WholeStageCodegen (23) + Exchange [ws_bill_customer_sk] #18 + WholeStageCodegen (17) Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] BroadcastHashJoin [ws_item_sk,item_sk] ColumnarToRow @@ -174,15 +169,15 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] InputAdapter ReusedExchange [item_sk] #4 InputAdapter - WholeStageCodegen (27) + WholeStageCodegen (20) Sort [c_customer_sk] Project [c_customer_sk] Filter [ssales] ReusedSubquery [tpcds_cmax] #3 HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [c_customer_sk,sum,isEmpty] #9 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt index 2635546e44..daff220647 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt @@ -1,44 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * ColumnarToRow (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometProject (18) + : : : : +- CometBroadcastHashJoin (17) + : : : : :- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (8) + : : : : +- CometBroadcastExchange (16) + : : : : +- CometProject (15) + : : : : +- CometFilter (14) + : : : : +- CometScan parquet spark_catalog.default.date_dim (13) + : : : +- CometBroadcastExchange (22) + : : : +- CometProject (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.date_dim (19) + : : +- ReusedExchange (25) + : +- CometBroadcastExchange (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.store (28) + +- CometBroadcastExchange (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.item (33) (1) Scan parquet spark_catalog.default.store_sales @@ -53,10 +55,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) -(6) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] +Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] +Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] +Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] +(7) CometProject Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] -(10) Scan parquet spark_catalog.default.catalog_sales +(8) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] @@ -93,206 +88,224 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] +Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] +Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] +Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] +(12) CometProject Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#19] +(13) Scan parquet spark_catalog.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_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None +(14) CometFilter +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] +(15) CometProject +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Arguments: [d_date_sk#19], [d_date_sk#19] + +(16) CometBroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: [d_date_sk#19] + +(17) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight + +(18) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] + +(19) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#22, d_year#23, d_moy#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(20) CometFilter +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) + +(21) CometProject +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Arguments: [d_date_sk#22], [d_date_sk#22] -(19) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#20] +(22) CometBroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: [d_date_sk#22] -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#22] +Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#20] +(24) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#22] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -(22) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#21] +(25) ReusedExchange [Reuses operator id: 22] +Output [1]: [d_date_sk#25] -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None +(26) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#21] +(27) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#25] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -(25) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +(28) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(26) CometFilter -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Condition : isnotnull(s_store_sk#22) +(29) CometFilter +Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] +Condition : isnotnull(s_store_sk#26) -(27) ColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +(30) CometBroadcastExchange +Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] +Arguments: [s_store_sk#26, s_store_id#27, s_store_name#28] -(28) BroadcastExchange -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(31) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] +Right output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] +Arguments: [ss_store_sk#3], [s_store_sk#26], Inner, BuildRight -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None +(32) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#26, s_store_id#27, s_store_name#28] +Arguments: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28], [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28] -(30) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#22, s_store_id#23, s_store_name#24] - -(31) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +(33) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Condition : isnotnull(i_item_sk#25) - -(33) ColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +(34) CometFilter +Input [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] +Condition : isnotnull(i_item_sk#29) -(34) BroadcastExchange -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(35) CometBroadcastExchange +Input [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] +Arguments: [i_item_sk#29, i_item_id#30, i_item_desc#31] -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#25] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28] +Right output [3]: [i_item_sk#29, i_item_id#30, i_item_desc#31] +Arguments: [ss_item_sk#1], [i_item_sk#29], Inner, BuildRight -(36) Project [codegen id : 8] -Output [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24, i_item_sk#25, i_item_id#26, i_item_desc#27] +(37) CometProject +Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_sk#29, i_item_id#30, i_item_desc#31] +Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31] -(37) HashAggregate [codegen id : 8] -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] -Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +(38) CometHashAggregate +Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31] +Keys [4]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28] Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum#28, sum#29, sum#30] -Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -(38) Exchange -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(39) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#32, sum#33, sum#34] + +(40) Exchange +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#32, sum#33, sum#34] +Arguments: hashpartitioning(i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(39) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +(41) HashAggregate [codegen id : 2] +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#32, sum#33, sum#34] +Keys [4]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#34, sum(UnscaledValue(sr_net_loss#11))#35, sum(UnscaledValue(cs_net_profit#16))#36] -Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#34,17,2) AS store_sales_profit#37, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#35,17,2) AS store_returns_loss#38, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#36,17,2) AS catalog_sales_profit#39] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#35, sum(UnscaledValue(sr_net_loss#11))#36, sum(UnscaledValue(cs_net_profit#16))#37] +Results [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#35,17,2) AS store_sales_profit#38, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#36,17,2) AS store_returns_loss#39, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#37,17,2) AS catalog_sales_profit#40] -(40) TakeOrderedAndProject -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] -Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] +(42) TakeOrderedAndProject +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40] +Arguments: 100, [i_item_id#30 ASC NULLS FIRST, i_item_desc#31 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(41) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#40, d_moy#41] +(43) Scan parquet spark_catalog.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_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] -Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 4)) AND (d_year#40 = 2001)) AND isnotnull(d_date_sk#19)) +(44) CometFilter +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) -(43) CometProject -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] +(45) CometProject +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(44) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(45) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#42, d_moy#43] +(48) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#22, d_year#23, d_moy#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] -Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 4)) AND (d_moy#43 <= 10)) AND (d_year#42 = 2001)) AND isnotnull(d_date_sk#20)) +(49) CometFilter +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(48) CometProject -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] -Arguments: [d_date_sk#20], [d_date_sk#20] +(50) CometProject +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Arguments: [d_date_sk#22], [d_date_sk#22] -(49) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] +(51) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#22] -(50) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(52) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index eda7f6b64d..74a5e87770 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (9) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] InputAdapter Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - 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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] + CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometProject [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,46 +31,33 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales CometProject [d_date_sk] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometBroadcastExchange #3 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #7 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange #8 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange #9 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index c90dcd0243..1369927fe5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -2,30 +2,30 @@ TakeOrderedAndProject (30) +- * HashAggregate (29) +- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) + +- * ColumnarToRow (27) + +- CometHashAggregate (26) + +- CometProject (25) + +- CometBroadcastHashJoin (24) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.date_dim (9) + : +- CometBroadcastExchange (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.item (15) + +- CometBroadcastExchange (23) +- CometProject (22) +- CometFilter (21) +- CometScan parquet spark_catalog.default.promotion (20) @@ -43,135 +43,136 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) ColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#10] -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_bill_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] +Right output [1]: [cd_demo_sk#10] +Arguments: [cs_bill_cdemo_sk#1], [cd_demo_sk#10], Inner, BuildRight -(10) Project [codegen id : 5] -Output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] +(8) CometProject Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] +Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] + +(9) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +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 + +(10) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) + +(11) CometProject +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14], [d_date_sk#14] -(11) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#14] +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] +Right output [1]: [d_date_sk#14] +Arguments: [cs_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight -(13) Project [codegen id : 5] -Output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] +(14) CometProject Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] +Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] -(14) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] +(15) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#16, i_item_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : isnotnull(i_item_sk#15) - -(16) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#15, i_item_id#16] +(16) CometFilter +Input [2]: [i_item_sk#16, i_item_id#17] +Condition : isnotnull(i_item_sk#16) -(17) BroadcastExchange -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [2]: [i_item_sk#16, i_item_id#17] +Arguments: [i_item_sk#16, i_item_id#17] -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] +Right output [2]: [i_item_sk#16, i_item_id#17] +Arguments: [cs_item_sk#2], [i_item_sk#16], Inner, BuildRight -(19) Project [codegen id : 5] -Output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16] -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#15, i_item_id#16] +(19) CometProject +Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#16, i_item_id#17] +Arguments: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17], [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] (20) Scan parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] +Output [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [Or(EqualTo(p_channel_email,N),EqualTo(p_channel_event,N)), IsNotNull(p_promo_sk)] ReadSchema: struct (21) CometFilter -Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] -Condition : (((p_channel_email#18 = N) OR (p_channel_event#19 = N)) AND isnotnull(p_promo_sk#17)) +Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] +Condition : (((p_channel_email#19 = N) OR (p_channel_event#20 = N)) AND isnotnull(p_promo_sk#18)) (22) CometProject -Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] -Arguments: [p_promo_sk#17], [p_promo_sk#17] +Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] +Arguments: [p_promo_sk#18], [p_promo_sk#18] -(23) ColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#17] +(23) CometBroadcastExchange +Input [1]: [p_promo_sk#18] +Arguments: [p_promo_sk#18] -(24) BroadcastExchange -Input [1]: [p_promo_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(24) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] +Right output [1]: [p_promo_sk#18] +Arguments: [cs_promo_sk#3], [p_promo_sk#18], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_promo_sk#3] -Right keys [1]: [p_promo_sk#17] -Join type: Inner -Join condition: None +(25) CometProject +Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17, p_promo_sk#18] +Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -(26) Project [codegen id : 5] -Output [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16] -Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16, p_promo_sk#17] - -(27) HashAggregate [codegen id : 5] -Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16] -Keys [1]: [i_item_id#16] +(26) CometHashAggregate +Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] +Keys [1]: [i_item_id#17] Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [8]: [sum#20, count#21, sum#22, count#23, sum#24, count#25, sum#26, count#27] -Results [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] + +(27) ColumnarToRow [codegen id : 1] +Input [9]: [i_item_id#17, sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] (28) Exchange -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [9]: [i_item_id#17, sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] +Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [1]: [i_item_id#16] +(29) HashAggregate [codegen id : 2] +Input [9]: [i_item_id#17, sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] +Keys [1]: [i_item_id#17] Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [4]: [avg(cs_quantity#4)#36, avg(UnscaledValue(cs_list_price#5))#37, avg(UnscaledValue(cs_coupon_amt#7))#38, avg(UnscaledValue(cs_sales_price#6))#39] -Results [5]: [i_item_id#16, avg(cs_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(cs_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(cs_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(cs_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] +Aggregate Attributes [4]: [avg(cs_quantity#4)#29, avg(UnscaledValue(cs_list_price#5))#30, avg(UnscaledValue(cs_coupon_amt#7))#31, avg(UnscaledValue(cs_sales_price#6))#32] +Results [5]: [i_item_id#17, avg(cs_quantity#4)#29 AS agg1#33, cast((avg(UnscaledValue(cs_list_price#5))#30 / 100.0) as decimal(11,6)) AS agg2#34, cast((avg(UnscaledValue(cs_coupon_amt#7))#31 / 100.0) as decimal(11,6)) AS agg3#35, cast((avg(UnscaledValue(cs_sales_price#6))#32 / 100.0) as decimal(11,6)) AS agg4#36] (30) TakeOrderedAndProject -Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] -Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] +Input [5]: [i_item_id#17, agg1#33, agg2#34, agg3#35, agg4#36] +Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#33, agg2#34, agg3#35, agg4#36] ===== Subqueries ===== @@ -184,18 +185,18 @@ BroadcastExchange (35) (31) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] 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 (32) CometFilter -Input [2]: [d_date_sk#14, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) (33) CometProject -Input [2]: [d_date_sk#14, d_year#44] +Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] (34) ColumnarToRow [codegen id : 1] @@ -203,6 +204,6 @@ Input [1]: [d_date_sk#14] (35) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index 7d38936244..cba306f68f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] + CometBroadcastHashJoin [cs_promo_sk,p_promo_sk] + CometProject [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -25,28 +25,18 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometBroadcastExchange #3 + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #5 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #6 + CometProject [p_promo_sk] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index e41077ed30..1692965c66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -2,31 +2,31 @@ TakeOrderedAndProject (30) +- * HashAggregate (29) +- Exchange (28) - +- * HashAggregate (27) - +- * Expand (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.store (14) - +- BroadcastExchange (23) - +- * ColumnarToRow (22) + +- * ColumnarToRow (27) + +- CometHashAggregate (26) + +- CometExpand (25) + +- CometProject (24) + +- CometBroadcastHashJoin (23) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.date_dim (9) + : +- CometBroadcastExchange (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.store (15) + +- CometBroadcastExchange (22) +- CometFilter (21) +- CometScan parquet spark_catalog.default.item (20) @@ -43,135 +43,136 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) ColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#10] -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [cd_demo_sk#10] +Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +(8) CometProject Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] + +(9) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct + +(10) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) + +(11) CometProject +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14], [d_date_sk#14] -(11) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#14] +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +(14) CometProject Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(14) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] +(15) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((isnotnull(s_state#16) AND (s_state#16 = TN)) AND isnotnull(s_store_sk#15)) - -(16) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#16] +(16) CometFilter +Input [2]: [s_store_sk#16, s_state#17] +Condition : ((isnotnull(s_state#17) AND (s_state#17 = TN)) AND isnotnull(s_store_sk#16)) -(17) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_state#17] +Arguments: [s_store_sk#16, s_state#17] -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [2]: [s_store_sk#16, s_state#17] +Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight -(19) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#16] +(19) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#17] +Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] (20) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_item_id#18] +Output [2]: [i_item_sk#18, i_item_id#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (21) CometFilter -Input [2]: [i_item_sk#17, i_item_id#18] -Condition : isnotnull(i_item_sk#17) +Input [2]: [i_item_sk#18, i_item_id#19] +Condition : isnotnull(i_item_sk#18) -(22) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#17, i_item_id#18] +(22) CometBroadcastExchange +Input [2]: [i_item_sk#18, i_item_id#19] +Arguments: [i_item_sk#18, i_item_id#19] -(23) BroadcastExchange -Input [2]: [i_item_sk#17, i_item_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] +Right output [2]: [i_item_sk#18, i_item_id#19] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(24) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#19] +Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17] -(25) Project [codegen id : 5] -Output [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, s_state#16] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16, i_item_sk#17, i_item_id#18] +(25) CometExpand +Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17] +Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#21, spark_grouping_id#22] -(26) Expand [codegen id : 5] -Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, s_state#16] -Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, s_state#16, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#20, spark_grouping_id#21] - -(27) HashAggregate [codegen id : 5] -Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#20, spark_grouping_id#21] -Keys [3]: [i_item_id#19, s_state#20, spark_grouping_id#21] +(26) CometHashAggregate +Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#21, spark_grouping_id#22] +Keys [3]: [i_item_id#20, s_state#21, spark_grouping_id#22] Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [8]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Results [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] + +(27) ColumnarToRow [codegen id : 1] +Input [11]: [i_item_id#20, s_state#21, spark_grouping_id#22, sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30] (28) Exchange -Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] -Arguments: hashpartitioning(i_item_id#19, s_state#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [11]: [i_item_id#20, s_state#21, spark_grouping_id#22, sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30] +Arguments: hashpartitioning(i_item_id#20, s_state#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 6] -Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] -Keys [3]: [i_item_id#19, s_state#20, spark_grouping_id#21] +(29) HashAggregate [codegen id : 2] +Input [11]: [i_item_id#20, s_state#21, spark_grouping_id#22, sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30] +Keys [3]: [i_item_id#20, s_state#21, spark_grouping_id#22] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#38, avg(UnscaledValue(ss_list_price#5))#39, avg(UnscaledValue(ss_coupon_amt#7))#40, avg(UnscaledValue(ss_sales_price#6))#41] -Results [7]: [i_item_id#19, s_state#20, cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) AS g_state#42, avg(ss_quantity#4)#38 AS agg1#43, cast((avg(UnscaledValue(ss_list_price#5))#39 / 100.0) as decimal(11,6)) AS agg2#44, cast((avg(UnscaledValue(ss_coupon_amt#7))#40 / 100.0) as decimal(11,6)) AS agg3#45, cast((avg(UnscaledValue(ss_sales_price#6))#41 / 100.0) as decimal(11,6)) AS agg4#46] +Aggregate Attributes [4]: [avg(ss_quantity#4)#31, avg(UnscaledValue(ss_list_price#5))#32, avg(UnscaledValue(ss_coupon_amt#7))#33, avg(UnscaledValue(ss_sales_price#6))#34] +Results [7]: [i_item_id#20, s_state#21, cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint) AS g_state#35, avg(ss_quantity#4)#31 AS agg1#36, cast((avg(UnscaledValue(ss_list_price#5))#32 / 100.0) as decimal(11,6)) AS agg2#37, cast((avg(UnscaledValue(ss_coupon_amt#7))#33 / 100.0) as decimal(11,6)) AS agg3#38, cast((avg(UnscaledValue(ss_sales_price#6))#34 / 100.0) as decimal(11,6)) AS agg4#39] (30) TakeOrderedAndProject -Input [7]: [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] -Arguments: 100, [i_item_id#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST], [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] +Input [7]: [i_item_id#20, s_state#21, g_state#35, agg1#36, agg2#37, agg3#38, agg4#39] +Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST], [i_item_id#20, s_state#21, g_state#35, agg1#36, agg2#37, agg3#38, agg4#39] ===== Subqueries ===== @@ -184,18 +185,18 @@ BroadcastExchange (35) (31) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#47] +Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (32) CometFilter -Input [2]: [d_date_sk#14, d_year#47] -Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) (33) CometProject -Input [2]: [d_date_sk#14, d_year#47] +Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] (34) ColumnarToRow [codegen id : 1] @@ -203,6 +204,6 @@ Input [1]: [d_date_sk#14] (35) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index 9d073ff67c..7fcbe967a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,s_state,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometExpand [i_item_id,s_state] [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state,spark_grouping_id] + CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -26,27 +26,17 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #3 + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #5 + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt index 522754cbc0..3c350ad07a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt @@ -1,44 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.item (31) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- Exchange (43) + +- * ColumnarToRow (42) + +- CometHashAggregate (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometProject (18) + : : : : +- CometBroadcastHashJoin (17) + : : : : :- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (8) + : : : : +- CometBroadcastExchange (16) + : : : : +- CometProject (15) + : : : : +- CometFilter (14) + : : : : +- CometScan parquet spark_catalog.default.date_dim (13) + : : : +- CometBroadcastExchange (22) + : : : +- CometProject (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.date_dim (19) + : : +- CometBroadcastExchange (28) + : : +- CometProject (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- CometBroadcastExchange (33) + : +- CometFilter (32) + : +- CometScan parquet spark_catalog.default.store (31) + +- CometBroadcastExchange (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.item (36) (1) Scan parquet spark_catalog.default.store_sales @@ -53,10 +58,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -(7) BroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] +Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] +(7) CometProject Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -(10) Scan parquet spark_catalog.default.catalog_sales +(8) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] @@ -93,177 +91,181 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] +Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +(12) CometProject Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] + +(13) Scan parquet spark_catalog.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_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#19] +(14) CometFilter +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None +(15) CometProject +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Arguments: [d_date_sk#19], [d_date_sk#19] + +(16) CometBroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: [d_date_sk#19] -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +(17) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight + +(18) CometProject Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -(19) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#20] +(19) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#22, d_year#23, d_moy#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(20) CometFilter +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] +(21) CometProject +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Arguments: [d_date_sk#22], [d_date_sk#22] -(22) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#21] +(22) CometBroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: [d_date_sk#22] -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#22] +Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] +(24) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#22] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -(25) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(26) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) + +(27) CometProject +Input [2]: [d_date_sk#25, d_year#26] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(28) CometBroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: [d_date_sk#25] + +(29) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight + +(30) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#25] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] + +(31) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(26) CometFilter -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Condition : isnotnull(s_store_sk#22) - -(27) ColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +(32) CometFilter +Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] +Condition : isnotnull(s_store_sk#27) -(28) BroadcastExchange -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(33) CometBroadcastExchange +Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] +Arguments: [s_store_sk#27, s_store_id#28, s_store_name#29] -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None +(34) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] +Right output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] +Arguments: [ss_store_sk#3], [s_store_sk#27], Inner, BuildRight -(30) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_store_id#23, s_store_name#24] +(35) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#27, s_store_id#28, s_store_name#29] +Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29] -(31) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +(36) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Condition : isnotnull(i_item_sk#25) - -(33) ColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +(37) CometFilter +Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] +Condition : isnotnull(i_item_sk#30) -(34) BroadcastExchange -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(38) CometBroadcastExchange +Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] +Arguments: [i_item_sk#30, i_item_id#31, i_item_desc#32] -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#25] -Join type: Inner -Join condition: None +(39) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29] +Right output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] +Arguments: [ss_item_sk#1], [i_item_sk#30], Inner, BuildRight -(36) Project [codegen id : 8] -Output [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] -Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24, i_item_sk#25, i_item_id#26, i_item_desc#27] +(40) CometProject +Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_sk#30, i_item_id#31, i_item_desc#32] +Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32] -(37) HashAggregate [codegen id : 8] -Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] -Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +(41) CometHashAggregate +Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32] +Keys [4]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29] Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] -Aggregate Attributes [3]: [sum#28, sum#29, sum#30] -Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -(38) Exchange -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(42) ColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum#33, sum#34, sum#35] -(39) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] -Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +(43) Exchange +Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum#33, sum#34, sum#35] +Arguments: hashpartitioning(i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(44) HashAggregate [codegen id : 2] +Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum#33, sum#34, sum#35] +Keys [4]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29] Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] -Aggregate Attributes [3]: [sum(ss_quantity#5)#34, sum(sr_return_quantity#11)#35, sum(cs_quantity#16)#36] -Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum(ss_quantity#5)#34 AS store_sales_quantity#37, sum(sr_return_quantity#11)#35 AS store_returns_quantity#38, sum(cs_quantity#16)#36 AS catalog_sales_quantity#39] +Aggregate Attributes [3]: [sum(ss_quantity#5)#36, sum(sr_return_quantity#11)#37, sum(cs_quantity#16)#38] +Results [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, sum(ss_quantity#5)#36 AS store_sales_quantity#39, sum(sr_return_quantity#11)#37 AS store_returns_quantity#40, sum(cs_quantity#16)#38 AS catalog_sales_quantity#41] -(40) TakeOrderedAndProject -Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] -Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] +(45) TakeOrderedAndProject +Input [7]: [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#39, store_returns_quantity#40, catalog_sales_quantity#41] +Arguments: 100, [i_item_id#31 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#28 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#31, i_item_desc#32, s_store_id#28, s_store_name#29, store_sales_quantity#39, store_returns_quantity#40, catalog_sales_quantity#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) - - -(41) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#40, d_moy#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] -Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 9)) AND (d_year#40 = 1999)) AND isnotnull(d_date_sk#19)) - -(43) CometProject -Input [3]: [d_date_sk#19, d_year#40, d_moy#41] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(44) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(45) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (50) +- * ColumnarToRow (49) +- CometProject (48) @@ -272,28 +274,28 @@ BroadcastExchange (50) (46) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#42, d_moy#43] +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_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct (47) CometFilter -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] -Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 9)) AND (d_moy#43 <= 12)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#20)) +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) (48) CometProject -Input [3]: [d_date_sk#20, d_year#42, d_moy#43] -Arguments: [d_date_sk#20], [d_date_sk#20] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +Arguments: [d_date_sk#19], [d_date_sk#19] (49) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] +Input [1]: [d_date_sk#19] (50) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 +Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (55) +- * ColumnarToRow (54) +- CometProject (53) @@ -302,25 +304,55 @@ BroadcastExchange (55) (51) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_year#44] +Output [3]: [d_date_sk#22, d_year#23, d_moy#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct (52) CometFilter -Input [2]: [d_date_sk#21, d_year#44] -Condition : (d_year#44 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) (53) CometProject -Input [2]: [d_date_sk#21, d_year#44] -Arguments: [d_date_sk#21], [d_date_sk#21] +Input [3]: [d_date_sk#22, d_year#23, d_moy#24] +Arguments: [d_date_sk#22], [d_date_sk#22] (54) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#21] +Input [1]: [d_date_sk#22] (55) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 +BroadcastExchange (60) ++- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) + + +(56) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(57) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) + +(58) CometProject +Input [2]: [d_date_sk#25, d_year#26] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(59) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] + +(60) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt index 68a127d357..2bcef6168a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - WholeStageCodegen (9) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] InputAdapter Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] + CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + CometBroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,53 +31,43 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales CometProject [d_date_sk] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 + CometBroadcastExchange #3 + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #7 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #8 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #9 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #10 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange #11 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt index e89cfe1fff..98092ef298 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt @@ -1,25 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (19) ++- * HashAggregate (18) + +- Exchange (17) + +- * ColumnarToRow (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- CometBroadcastExchange (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.store_sales (4) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.item (9) (1) Scan parquet spark_catalog.default.date_dim @@ -37,10 +35,7 @@ Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) Input [3]: [d_date_sk#1, d_year#2, d_moy#3] Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] -(4) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#1, d_year#2] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -48,78 +43,71 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Condition : isnotnull(ss_item_sk#4) -(7) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [d_date_sk#1, d_year#2] +Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight -(10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +(8) CometProject Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -(11) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) -(13) CometProject +(11) CometProject Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9], [i_item_sk#7, i_brand_id#8, i_brand#9] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9] -(15) BroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#4] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight -(17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +(14) CometProject Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(18) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#11] -Results [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -(19) Exchange -Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(16) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#11] + +(17) Exchange +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#11] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] -Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] +(18) HashAggregate [codegen id : 2] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#11] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum_agg#16] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#12] +Results [4]: [d_year#2, i_brand_id#8 AS brand_id#13, i_brand#9 AS brand#14, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#12,17,2) AS sum_agg#15] -(21) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#14, brand#15, sum_agg#16] -Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, sum_agg#16] +(19) TakeOrderedAndProject +Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] +Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt index 3946c0cd8f..cc9c4edf3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt @@ -1,31 +1,23 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 - 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_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_year] CometFilter [d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index 2a4cc2422b..55c49d828f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt @@ -1,53 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (33) - : : +- * Filter (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * HashAggregate (28) - : : +- Exchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.web_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : +- BroadcastExchange (39) - : +- * ColumnarToRow (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.customer (36) - +- BroadcastExchange (46) - +- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.customer_address (42) +TakeOrderedAndProject (51) ++- * Project (50) + +- * BroadcastHashJoin Inner BuildRight (49) + :- * Project (43) + : +- * BroadcastHashJoin Inner BuildRight (42) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- Exchange (16) + : : : +- * ColumnarToRow (15) + : : : +- CometHashAggregate (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.customer_address (9) + : : +- BroadcastExchange (35) + : : +- * Filter (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * HashAggregate (31) + : : +- * HashAggregate (30) + : : +- Exchange (29) + : : +- * ColumnarToRow (28) + : : +- CometHashAggregate (27) + : : +- CometProject (26) + : : +- CometBroadcastHashJoin (25) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.web_returns (19) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : +- BroadcastExchange (41) + : +- * ColumnarToRow (40) + : +- CometFilter (39) + : +- CometScan parquet spark_catalog.default.customer (38) + +- BroadcastExchange (48) + +- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.customer_address (44) (1) Scan parquet spark_catalog.default.web_returns @@ -62,73 +64,82 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 54] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_year#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] + +(7) CometBroadcastHashJoin +Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight -(6) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] +(8) CometProject Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] +Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -(7) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_state#8] +(9) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#8, ca_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] ReadSchema: struct -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_state#8] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(ca_state#8)) +(10) CometFilter +Input [2]: [ca_address_sk#8, ca_state#9] +Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_state#9)) -(9) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#7, ca_state#8] +(11) CometBroadcastExchange +Input [2]: [ca_address_sk#8, ca_state#9] +Arguments: [ca_address_sk#8, ca_state#9] -(10) BroadcastExchange -Input [2]: [ca_address_sk#7, ca_state#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] +Right output [2]: [ca_address_sk#8, ca_state#9] +Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#8] +(13) CometProject +Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#9] +Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -(13) HashAggregate [codegen id : 3] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] -Keys [2]: [wr_returning_customer_sk#1, ca_state#8] +(14) CometHashAggregate +Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] +Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#9] -Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] -(14) Exchange -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(15) ColumnarToRow [codegen id : 1] +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#10] + +(16) Exchange +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#10] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 11] -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#8] +(17) HashAggregate [codegen id : 7] +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#10] +Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#11] -Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] +Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#9 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] -(16) Filter [codegen id : 11] +(18) Filter [codegen id : 7] Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] Condition : isnotnull(ctr_total_return#14) -(17) Scan parquet spark_catalog.default.web_returns +(19) Scan parquet spark_catalog.default.web_returns Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] @@ -136,189 +147,185 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(18) CometFilter +(20) CometFilter Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Condition : isnotnull(wr_returning_addr_sk#2) -(19) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] - -(20) ReusedExchange [Reuses operator id: 54] +(21) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#6] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight -(22) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] +(23) CometProject Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] +Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#7, ca_state#8] +(24) ReusedExchange [Reuses operator id: 11] +Output [2]: [ca_address_sk#8, ca_state#9] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] +Right output [2]: [ca_address_sk#8, ca_state#9] +Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight -(25) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#8] +(26) CometProject +Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#9] +Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -(26) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] -Keys [2]: [wr_returning_customer_sk#1, ca_state#8] +(27) CometHashAggregate +Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] +Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#16] -Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] -(27) Exchange -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(28) ColumnarToRow [codegen id : 2] +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#16] -(28) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#17] -Keys [2]: [wr_returning_customer_sk#1, ca_state#8] +(29) Exchange +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#16] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(30) HashAggregate [codegen id : 3] +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#16] +Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#11] -Results [2]: [ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] +Results [2]: [ca_state#9 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] -(29) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 3] Input [2]: [ctr_state#13, ctr_total_return#14] Keys [1]: [ctr_state#13] Functions [1]: [partial_avg(ctr_total_return#14)] -Aggregate Attributes [2]: [sum#18, count#19] -Results [3]: [ctr_state#13, sum#20, count#21] +Aggregate Attributes [2]: [sum#17, count#18] +Results [3]: [ctr_state#13, sum#19, count#20] -(30) Exchange -Input [3]: [ctr_state#13, sum#20, count#21] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(32) Exchange +Input [3]: [ctr_state#13, sum#19, count#20] +Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(31) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#13, sum#20, count#21] +(33) HashAggregate [codegen id : 4] +Input [3]: [ctr_state#13, sum#19, count#20] Keys [1]: [ctr_state#13] Functions [1]: [avg(ctr_total_return#14)] -Aggregate Attributes [1]: [avg(ctr_total_return#14)#22] -Results [2]: [(avg(ctr_total_return#14)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#13 AS ctr_state#13#24] +Aggregate Attributes [1]: [avg(ctr_total_return#14)#21] +Results [2]: [(avg(ctr_total_return#14)#21 * 1.2) AS (avg(ctr_total_return) * 1.2)#22, ctr_state#13 AS ctr_state#13#23] -(32) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) +(34) Filter [codegen id : 4] +Input [2]: [(avg(ctr_total_return) * 1.2)#22, ctr_state#13#23] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#22) -(33) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] +(35) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#22, ctr_state#13#23] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_state#13] -Right keys [1]: [ctr_state#13#24] +Right keys [1]: [ctr_state#13#23] Join type: Inner -Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) +Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#22) -(35) Project [codegen id : 11] +(37) Project [codegen id : 7] Output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] +Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#22, ctr_state#13#23] -(36) Scan parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] +(38) Scan parquet spark_catalog.default.customer +Output [14]: [c_customer_sk#24, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_day#31, c_birth_month#32, c_birth_year#33, c_birth_country#34, c_login#35, c_email_address#36, c_last_review_date#37] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(37) CometFilter -Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) +(39) CometFilter +Input [14]: [c_customer_sk#24, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_day#31, c_birth_month#32, c_birth_year#33, c_birth_country#34, c_login#35, c_email_address#36, c_last_review_date#37] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#26)) -(38) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] +(40) ColumnarToRow [codegen id : 5] +Input [14]: [c_customer_sk#24, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_day#31, c_birth_month#32, c_birth_year#33, c_birth_country#34, c_login#35, c_email_address#36, c_last_review_date#37] -(39) BroadcastExchange -Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(41) BroadcastExchange +Input [14]: [c_customer_sk#24, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_day#31, c_birth_month#32, c_birth_year#33, c_birth_country#34, c_login#35, c_email_address#36, c_last_review_date#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(40) BroadcastHashJoin [codegen id : 11] +(42) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_customer_sk#12] -Right keys [1]: [c_customer_sk#25] +Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(41) Project [codegen id : 11] -Output [14]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] -Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] +(43) Project [codegen id : 7] +Output [14]: [ctr_total_return#14, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_day#31, c_birth_month#32, c_birth_year#33, c_birth_country#34, c_login#35, c_email_address#36, c_last_review_date#37] +Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#24, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_day#31, c_birth_month#32, c_birth_year#33, c_birth_country#34, c_login#35, c_email_address#36, c_last_review_date#37] -(42) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#39, ca_state#40] +(44) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#38, ca_state#39] 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 -(43) CometFilter -Input [2]: [ca_address_sk#39, ca_state#40] -Condition : ((isnotnull(ca_state#40) AND (ca_state#40 = GA)) AND isnotnull(ca_address_sk#39)) +(45) CometFilter +Input [2]: [ca_address_sk#38, ca_state#39] +Condition : ((isnotnull(ca_state#39) AND (ca_state#39 = GA)) AND isnotnull(ca_address_sk#38)) -(44) CometProject -Input [2]: [ca_address_sk#39, ca_state#40] -Arguments: [ca_address_sk#39], [ca_address_sk#39] +(46) CometProject +Input [2]: [ca_address_sk#38, ca_state#39] +Arguments: [ca_address_sk#38], [ca_address_sk#38] -(45) ColumnarToRow [codegen id : 10] -Input [1]: [ca_address_sk#39] +(47) ColumnarToRow [codegen id : 6] +Input [1]: [ca_address_sk#38] -(46) BroadcastExchange -Input [1]: [ca_address_sk#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(48) BroadcastExchange +Input [1]: [ca_address_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#27] -Right keys [1]: [ca_address_sk#39] +(49) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#26] +Right keys [1]: [ca_address_sk#38] Join type: Inner Join condition: None -(48) Project [codegen id : 11] -Output [13]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] -Input [15]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ca_address_sk#39] +(50) Project [codegen id : 7] +Output [13]: [c_customer_id#25, c_salutation#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_day#31, c_birth_month#32, c_birth_year#33, c_birth_country#34, c_login#35, c_email_address#36, c_last_review_date#37, ctr_total_return#14] +Input [15]: [ctr_total_return#14, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_day#31, c_birth_month#32, c_birth_year#33, c_birth_country#34, c_login#35, c_email_address#36, c_last_review_date#37, ca_address_sk#38] -(49) TakeOrderedAndProject -Input [13]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] -Arguments: 100, [c_customer_id#26 ASC NULLS FIRST, c_salutation#28 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, c_last_name#30 ASC NULLS FIRST, c_preferred_cust_flag#31 ASC NULLS FIRST, c_birth_day#32 ASC NULLS FIRST, c_birth_month#33 ASC NULLS FIRST, c_birth_year#34 ASC NULLS FIRST, c_birth_country#35 ASC NULLS FIRST, c_login#36 ASC NULLS FIRST, c_email_address#37 ASC NULLS FIRST, c_last_review_date#38 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38, ctr_total_return#14] +(51) TakeOrderedAndProject +Input [13]: [c_customer_id#25, c_salutation#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_day#31, c_birth_month#32, c_birth_year#33, c_birth_country#34, c_login#35, c_email_address#36, c_last_review_date#37, ctr_total_return#14] +Arguments: 100, [c_customer_id#25 ASC NULLS FIRST, c_salutation#27 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST, c_last_name#29 ASC NULLS FIRST, c_preferred_cust_flag#30 ASC NULLS FIRST, c_birth_day#31 ASC NULLS FIRST, c_birth_month#32 ASC NULLS FIRST, c_birth_year#33 ASC NULLS FIRST, c_birth_country#34 ASC NULLS FIRST, c_login#35 ASC NULLS FIRST, c_email_address#36 ASC NULLS FIRST, c_last_review_date#37 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#25, c_salutation#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_day#31, c_birth_month#32, c_birth_year#33, c_birth_country#34, c_login#35, c_email_address#36, c_last_review_date#37, ctr_total_return#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +BroadcastExchange (56) ++- * ColumnarToRow (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) -(50) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#41] +(52) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_year#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter -Input [2]: [d_date_sk#6, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#6)) +(53) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) -(52) CometProject -Input [2]: [d_date_sk#6, d_year#41] +(54) CometProject +Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(53) ColumnarToRow [codegen id : 1] +(55) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(54) BroadcastExchange +(56) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 17 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt index c70f33da33..50472502e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (11) + WholeStageCodegen (7) Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] @@ -10,14 +10,14 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] InputAdapter Exchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -28,52 +28,48 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) + BroadcastExchange #5 + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] InputAdapter - Exchange [ctr_state] #5 - WholeStageCodegen (7) + Exchange [ctr_state] #6 + WholeStageCodegen (3) HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] InputAdapter - Exchange [wr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [wr_returning_customer_sk,ca_state] #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] CometFilter [wr_returning_addr_sk] CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) + BroadcastExchange #8 + WholeStageCodegen (5) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) + BroadcastExchange #9 + WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [ca_address_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index e20d454864..d63ab73567 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt @@ -1,94 +1,99 @@ == Physical Plan == -* Sort (90) -+- Exchange (89) - +- * Project (88) - +- * BroadcastHashJoin Inner BuildRight (87) - :- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * BroadcastHashJoin Inner BuildRight (29) - : : : : :- * HashAggregate (15) - : : : : : +- Exchange (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (6) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- ReusedExchange (4) - : : : : : +- BroadcastExchange (10) - : : : : : +- * ColumnarToRow (9) - : : : : : +- CometFilter (8) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : : : +- BroadcastExchange (28) - : : : : +- * HashAggregate (27) - : : : : +- Exchange (26) - : : : : +- * HashAggregate (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : :- * Project (21) - : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : :- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (16) - : : : : : +- ReusedExchange (19) - : : : : +- ReusedExchange (22) - : : : +- BroadcastExchange (42) - : : : +- * HashAggregate (41) - : : : +- Exchange (40) - : : : +- * HashAggregate (39) - : : : +- * Project (38) - : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : :- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * ColumnarToRow (32) - : : : : : +- CometFilter (31) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (33) - : : : +- ReusedExchange (36) - : : +- BroadcastExchange (57) - : : +- * HashAggregate (56) - : : +- Exchange (55) - : : +- * HashAggregate (54) - : : +- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * ColumnarToRow (47) - : : : : +- CometFilter (46) - : : : : +- CometScan parquet spark_catalog.default.web_sales (45) - : : : +- ReusedExchange (48) - : : +- ReusedExchange (51) - : +- BroadcastExchange (71) - : +- * HashAggregate (70) - : +- Exchange (69) - : +- * HashAggregate (68) - : +- * Project (67) - : +- * BroadcastHashJoin Inner BuildRight (66) - : :- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * ColumnarToRow (61) - : : : +- CometFilter (60) - : : : +- CometScan parquet spark_catalog.default.web_sales (59) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- BroadcastExchange (86) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * BroadcastHashJoin Inner BuildRight (81) - :- * Project (79) - : +- * BroadcastHashJoin Inner BuildRight (78) - : :- * ColumnarToRow (76) - : : +- CometFilter (75) - : : +- CometScan parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (77) - +- ReusedExchange (80) +* Sort (95) ++- Exchange (94) + +- * Project (93) + +- * BroadcastHashJoin Inner BuildRight (92) + :- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : :- * BroadcastHashJoin Inner BuildRight (32) + : : : : :- * HashAggregate (16) + : : : : : +- Exchange (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- CometHashAggregate (13) + : : : : : +- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (8) + : : : : +- BroadcastExchange (31) + : : : : +- * HashAggregate (30) + : : : : +- Exchange (29) + : : : : +- * ColumnarToRow (28) + : : : : +- CometHashAggregate (27) + : : : : +- CometProject (26) + : : : : +- CometBroadcastHashJoin (25) + : : : : :- CometProject (23) + : : : : : +- CometBroadcastHashJoin (22) + : : : : : :- CometFilter (18) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (17) + : : : : : +- CometBroadcastExchange (21) + : : : : : +- CometFilter (20) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (19) + : : : : +- ReusedExchange (24) + : : : +- BroadcastExchange (47) + : : : +- * HashAggregate (46) + : : : +- Exchange (45) + : : : +- * ColumnarToRow (44) + : : : +- CometHashAggregate (43) + : : : +- CometProject (42) + : : : +- CometBroadcastHashJoin (41) + : : : :- CometProject (39) + : : : : +- CometBroadcastHashJoin (38) + : : : : :- CometFilter (34) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (33) + : : : : +- CometBroadcastExchange (37) + : : : : +- CometFilter (36) + : : : : +- CometScan parquet spark_catalog.default.date_dim (35) + : : : +- ReusedExchange (40) + : : +- BroadcastExchange (62) + : : +- * HashAggregate (61) + : : +- Exchange (60) + : : +- * ColumnarToRow (59) + : : +- CometHashAggregate (58) + : : +- CometProject (57) + : : +- CometBroadcastHashJoin (56) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometFilter (51) + : : : : +- CometScan parquet spark_catalog.default.web_sales (50) + : : : +- ReusedExchange (52) + : : +- ReusedExchange (55) + : +- BroadcastExchange (76) + : +- * HashAggregate (75) + : +- Exchange (74) + : +- * ColumnarToRow (73) + : +- CometHashAggregate (72) + : +- CometProject (71) + : +- CometBroadcastHashJoin (70) + : :- CometProject (68) + : : +- CometBroadcastHashJoin (67) + : : :- CometFilter (65) + : : : +- CometScan parquet spark_catalog.default.web_sales (64) + : : +- ReusedExchange (66) + : +- ReusedExchange (69) + +- BroadcastExchange (91) + +- * HashAggregate (90) + +- Exchange (89) + +- * ColumnarToRow (88) + +- CometHashAggregate (87) + +- CometProject (86) + +- CometBroadcastHashJoin (85) + :- CometProject (83) + : +- CometBroadcastHashJoin (82) + : :- CometFilter (80) + : : +- CometScan parquet spark_catalog.default.web_sales (79) + : +- ReusedExchange (81) + +- ReusedExchange (84) (1) Scan parquet spark_catalog.default.store_sales @@ -103,514 +108,523 @@ ReadSchema: struct Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_addr_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(4) ReusedExchange [Reuses operator id: 94] +(3) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(4) CometFilter +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(6) Project [codegen id : 3] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] +(5) CometBroadcastExchange +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Arguments: [d_date_sk#5, d_year#6, d_qoy#7] + +(6) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Right output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(7) CometProject Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] +Arguments: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7], [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] -(7) Scan parquet spark_catalog.default.customer_address +(8) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#8, ca_county#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] ReadSchema: struct -(8) CometFilter +(9) CometFilter Input [2]: [ca_address_sk#8, ca_county#9] Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) -(9) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [2]: [ca_address_sk#8, ca_county#9] +Arguments: [ca_address_sk#8, ca_county#9] -(10) BroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] +Right output [2]: [ca_address_sk#8, ca_county#9] +Arguments: [ss_addr_sk#1], [ca_address_sk#8], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#8] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] +(12) CometProject Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] +Arguments: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9], [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -(13) HashAggregate [codegen id : 3] +(13) CometHashAggregate Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#10] -Results [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -(14) Exchange -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(14) ColumnarToRow [codegen id : 1] +Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -(15) HashAggregate [codegen id : 24] -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] +(15) Exchange +Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] +Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(16) HashAggregate [codegen id : 12] +Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] -Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS store_sales#13] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#11] +Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS store_sales#12] -(16) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] +(17) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(17) CometFilter -Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -Condition : isnotnull(ss_addr_sk#14) +(18) CometFilter +Input [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] +Condition : isnotnull(ss_addr_sk#13) + +(19) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] +(20) CometFilter +Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] +Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) -(19) ReusedExchange [Reuses operator id: 98] -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +(21) CometBroadcastExchange +Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] +Arguments: [d_date_sk#17, d_year#18, d_qoy#19] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#16] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] +Right output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight -(21) Project [codegen id : 6] -Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] -Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] +(23) CometProject +Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_year#18, d_qoy#19] +Arguments: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19], [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19] -(22) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#21, ca_county#22] +(24) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#20, ca_county#21] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#14] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19] +Right output [2]: [ca_address_sk#20, ca_county#21] +Arguments: [ss_addr_sk#13], [ca_address_sk#20], Inner, BuildRight -(24) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] - -(25) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -Keys [3]: [ca_county#22, d_qoy#20, d_year#19] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] -Aggregate Attributes [1]: [sum#23] -Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] - -(26) Exchange -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 7] -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Keys [3]: [ca_county#22, d_qoy#20, d_year#19] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#15))#12] -Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#12,17,2) AS store_sales#25] - -(28) BroadcastExchange -Input [2]: [ca_county#22, store_sales#25] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#22] -Join type: Inner -Join condition: None +(26) CometProject +Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_address_sk#20, ca_county#21] +Arguments: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21], [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] -(30) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct +(27) CometHashAggregate +Input [4]: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] +Keys [3]: [ca_county#21, d_qoy#19, d_year#18] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#14))] -(31) CometFilter -Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : isnotnull(ss_addr_sk#26) +(28) ColumnarToRow [codegen id : 2] +Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] -(32) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +(29) Exchange +Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] +Arguments: hashpartitioning(ca_county#21, d_qoy#19, d_year#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(33) ReusedExchange [Reuses operator id: 102] -Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] +(30) HashAggregate [codegen id : 3] +Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] +Keys [3]: [ca_county#21, d_qoy#19, d_year#18] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#14))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#14))#11] +Results [2]: [ca_county#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#14))#11,17,2) AS store_sales#23] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#30] +(31) BroadcastExchange +Input [2]: [ca_county#21, store_sales#23] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(32) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [ca_county#9] +Right keys [1]: [ca_county#21] Join type: Inner Join condition: None -(35) Project [codegen id : 10] -Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] -Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] +(33) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PushedFilters: [IsNotNull(ss_addr_sk)] +ReadSchema: struct -(36) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#33, ca_county#34] +(34) CometFilter +Input [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_addr_sk#24) -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#26] -Right keys [1]: [ca_address_sk#33] -Join type: Inner -Join condition: None +(35) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(38) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] - -(39) HashAggregate [codegen id : 10] -Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -Keys [3]: [ca_county#34, d_qoy#32, d_year#31] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#35] -Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] - -(40) Exchange -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(41) HashAggregate [codegen id : 11] -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Keys [3]: [ca_county#34, d_qoy#32, d_year#31] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#12] -Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#12,17,2) AS store_sales#37] - -(42) BroadcastExchange -Input [2]: [ca_county#34, store_sales#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(43) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#22] -Right keys [1]: [ca_county#34] +(36) CometFilter +Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] +Condition : ((((isnotnull(d_qoy#30) AND isnotnull(d_year#29)) AND (d_qoy#30 = 3)) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) + +(37) CometBroadcastExchange +Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] +Arguments: [d_date_sk#28, d_year#29, d_qoy#30] + +(38) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Right output [3]: [d_date_sk#28, d_year#29, d_qoy#30] +Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight + +(39) CometProject +Input [6]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29, d_qoy#30] +Arguments: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30], [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30] + +(40) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#31, ca_county#32] + +(41) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30] +Right output [2]: [ca_address_sk#31, ca_county#32] +Arguments: [ss_addr_sk#24], [ca_address_sk#31], Inner, BuildRight + +(42) CometProject +Input [6]: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_address_sk#31, ca_county#32] +Arguments: [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32], [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32] + +(43) CometHashAggregate +Input [4]: [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32] +Keys [3]: [ca_county#32, d_qoy#30, d_year#29] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#25))] + +(44) ColumnarToRow [codegen id : 4] +Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] + +(45) Exchange +Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] +Arguments: hashpartitioning(ca_county#32, d_qoy#30, d_year#29, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(46) HashAggregate [codegen id : 5] +Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] +Keys [3]: [ca_county#32, d_qoy#30, d_year#29] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#25))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#25))#11] +Results [2]: [ca_county#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#25))#11,17,2) AS store_sales#34] + +(47) BroadcastExchange +Input [2]: [ca_county#32, store_sales#34] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(48) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [ca_county#21] +Right keys [1]: [ca_county#32] Join type: Inner Join condition: None -(44) Project [codegen id : 24] -Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] -Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] +(49) Project [codegen id : 12] +Output [5]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34] +Input [7]: [ca_county#9, d_year#6, store_sales#12, ca_county#21, store_sales#23, ca_county#32, store_sales#34] -(45) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(50) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#41)] +PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(46) CometFilter -Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_addr_sk#38) +(51) CometFilter +Input [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] +Condition : isnotnull(ws_bill_addr_sk#35) -(47) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(52) ReusedExchange [Reuses operator id: 5] +Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] -(48) ReusedExchange [Reuses operator id: 94] -Output [3]: [d_date_sk#42, d_year#43, d_qoy#44] +(53) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] +Right output [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Arguments: [ws_sold_date_sk#37], [d_date_sk#39], Inner, BuildRight -(49) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#42] -Join type: Inner -Join condition: None +(54) CometProject +Input [6]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37, d_date_sk#39, d_year#40, d_qoy#41] +Arguments: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41], [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41] -(50) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44] -Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#42, d_year#43, d_qoy#44] +(55) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#42, ca_county#43] -(51) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#45, ca_county#46] +(56) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41] +Right output [2]: [ca_address_sk#42, ca_county#43] +Arguments: [ws_bill_addr_sk#35], [ca_address_sk#42], Inner, BuildRight -(52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#45] -Join type: Inner -Join condition: None +(57) CometProject +Input [6]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_address_sk#42, ca_county#43] +Arguments: [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43], [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43] + +(58) CometHashAggregate +Input [4]: [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43] +Keys [3]: [ca_county#43, d_qoy#41, d_year#40] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#36))] + +(59) ColumnarToRow [codegen id : 6] +Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] + +(60) Exchange +Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] +Arguments: hashpartitioning(ca_county#43, d_qoy#41, d_year#40, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(53) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] -Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_address_sk#45, ca_county#46] - -(54) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] -Keys [3]: [ca_county#46, d_qoy#44, d_year#43] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#47] -Results [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] - -(55) Exchange -Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] -Arguments: hashpartitioning(ca_county#46, d_qoy#44, d_year#43, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(56) HashAggregate [codegen id : 15] -Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] -Keys [3]: [ca_county#46, d_qoy#44, d_year#43] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#49] -Results [2]: [ca_county#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#49,17,2) AS web_sales#50] - -(57) BroadcastExchange -Input [2]: [ca_county#46, web_sales#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(58) BroadcastHashJoin [codegen id : 24] +(61) HashAggregate [codegen id : 7] +Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] +Keys [3]: [ca_county#43, d_qoy#41, d_year#40] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#36))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#36))#45] +Results [2]: [ca_county#43, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#36))#45,17,2) AS web_sales#46] + +(62) BroadcastExchange +Input [2]: [ca_county#43, web_sales#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] + +(63) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#46] +Right keys [1]: [ca_county#43] Join type: Inner Join condition: None -(59) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] +(64) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ws_sold_date_sk#49), dynamicpruningexpression(ws_sold_date_sk#49 IN dynamicpruning#50)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(60) CometFilter -Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_bill_addr_sk#51) +(65) CometFilter +Input [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] +Condition : isnotnull(ws_bill_addr_sk#47) -(61) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] +(66) ReusedExchange [Reuses operator id: 21] +Output [3]: [d_date_sk#51, d_year#52, d_qoy#53] -(62) ReusedExchange [Reuses operator id: 98] -Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] +(67) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] +Right output [3]: [d_date_sk#51, d_year#52, d_qoy#53] +Arguments: [ws_sold_date_sk#49], [d_date_sk#51], Inner, BuildRight -(63) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#53] -Right keys [1]: [d_date_sk#55] -Join type: Inner -Join condition: None +(68) CometProject +Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49, d_date_sk#51, d_year#52, d_qoy#53] +Arguments: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53], [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53] -(64) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57] -Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56, d_qoy#57] +(69) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#54, ca_county#55] -(65) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#58, ca_county#59] +(70) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53] +Right output [2]: [ca_address_sk#54, ca_county#55] +Arguments: [ws_bill_addr_sk#47], [ca_address_sk#54], Inner, BuildRight -(66) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#51] -Right keys [1]: [ca_address_sk#58] -Join type: Inner -Join condition: None +(71) CometProject +Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_address_sk#54, ca_county#55] +Arguments: [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55], [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55] + +(72) CometHashAggregate +Input [4]: [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55] +Keys [3]: [ca_county#55, d_qoy#53, d_year#52] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#48))] + +(73) ColumnarToRow [codegen id : 8] +Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] + +(74) Exchange +Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] +Arguments: hashpartitioning(ca_county#55, d_qoy#53, d_year#52, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(67) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] -Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_address_sk#58, ca_county#59] - -(68) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] -Keys [3]: [ca_county#59, d_qoy#57, d_year#56] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#52))] -Aggregate Attributes [1]: [sum#60] -Results [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] - -(69) Exchange -Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] -Arguments: hashpartitioning(ca_county#59, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(70) HashAggregate [codegen id : 19] -Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] -Keys [3]: [ca_county#59, d_qoy#57, d_year#56] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#52))#49] -Results [2]: [ca_county#59, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#52))#49,17,2) AS web_sales#62] - -(71) BroadcastExchange -Input [2]: [ca_county#59, web_sales#62] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] - -(72) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#46] -Right keys [1]: [ca_county#59] +(75) HashAggregate [codegen id : 9] +Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] +Keys [3]: [ca_county#55, d_qoy#53, d_year#52] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#48))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#48))#45] +Results [2]: [ca_county#55, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#48))#45,17,2) AS web_sales#57] + +(76) BroadcastExchange +Input [2]: [ca_county#55, web_sales#57] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=9] + +(77) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [ca_county#43] +Right keys [1]: [ca_county#55] Join type: Inner -Join condition: (CASE WHEN (web_sales#50 > 0.00) THEN (web_sales#62 / web_sales#50) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) +Join condition: (CASE WHEN (web_sales#46 > 0.00) THEN (web_sales#57 / web_sales#46) END > CASE WHEN (store_sales#12 > 0.00) THEN (store_sales#23 / store_sales#12) END) -(73) Project [codegen id : 24] -Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62] -Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, ca_county#59, web_sales#62] +(78) Project [codegen id : 12] +Output [8]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, web_sales#57] +Input [9]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, ca_county#55, web_sales#57] -(74) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +(79) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#66)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(75) CometFilter -Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_bill_addr_sk#63) +(80) CometFilter +Input [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] +Condition : isnotnull(ws_bill_addr_sk#58) -(76) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +(81) ReusedExchange [Reuses operator id: 37] +Output [3]: [d_date_sk#62, d_year#63, d_qoy#64] -(77) ReusedExchange [Reuses operator id: 102] -Output [3]: [d_date_sk#67, d_year#68, d_qoy#69] +(82) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] +Right output [3]: [d_date_sk#62, d_year#63, d_qoy#64] +Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight -(78) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#67] -Join type: Inner -Join condition: None +(83) CometProject +Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63, d_qoy#64] +Arguments: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64], [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64] -(79) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69] -Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68, d_qoy#69] +(84) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#65, ca_county#66] -(80) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#70, ca_county#71] +(85) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64] +Right output [2]: [ca_address_sk#65, ca_county#66] +Arguments: [ws_bill_addr_sk#58], [ca_address_sk#65], Inner, BuildRight -(81) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#63] -Right keys [1]: [ca_address_sk#70] -Join type: Inner -Join condition: None +(86) CometProject +Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_address_sk#65, ca_county#66] +Arguments: [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66], [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66] -(82) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] -Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_address_sk#70, ca_county#71] - -(83) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] -Keys [3]: [ca_county#71, d_qoy#69, d_year#68] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] -Aggregate Attributes [1]: [sum#72] -Results [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] - -(84) Exchange -Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] -Arguments: hashpartitioning(ca_county#71, d_qoy#69, d_year#68, 5), ENSURE_REQUIREMENTS, [plan_id=11] - -(85) HashAggregate [codegen id : 23] -Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] -Keys [3]: [ca_county#71, d_qoy#69, d_year#68] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#64))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#64))#49] -Results [2]: [ca_county#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#49,17,2) AS web_sales#74] - -(86) BroadcastExchange -Input [2]: [ca_county#71, web_sales#74] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] - -(87) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#46] -Right keys [1]: [ca_county#71] -Join type: Inner -Join condition: (CASE WHEN (web_sales#62 > 0.00) THEN (web_sales#74 / web_sales#62) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) +(87) CometHashAggregate +Input [4]: [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66] +Keys [3]: [ca_county#66, d_qoy#64, d_year#63] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#59))] -(88) Project [codegen id : 24] -Output [6]: [ca_county#9, d_year#6, (web_sales#62 / web_sales#50) AS web_q1_q2_increase#75, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#76, (web_sales#74 / web_sales#62) AS web_q2_q3_increase#77, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#78] -Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62, ca_county#71, web_sales#74] +(88) ColumnarToRow [codegen id : 10] +Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] (89) Exchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] +Arguments: hashpartitioning(ca_county#66, d_qoy#64, d_year#63, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(90) HashAggregate [codegen id : 11] +Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] +Keys [3]: [ca_county#66, d_qoy#64, d_year#63] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#59))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#59))#45] +Results [2]: [ca_county#66, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#45,17,2) AS web_sales#68] + +(91) BroadcastExchange +Input [2]: [ca_county#66, web_sales#68] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] + +(92) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [ca_county#43] +Right keys [1]: [ca_county#66] +Join type: Inner +Join condition: (CASE WHEN (web_sales#57 > 0.00) THEN (web_sales#68 / web_sales#57) END > CASE WHEN (store_sales#23 > 0.00) THEN (store_sales#34 / store_sales#23) END) + +(93) Project [codegen id : 12] +Output [6]: [ca_county#9, d_year#6, (web_sales#57 / web_sales#46) AS web_q1_q2_increase#69, (store_sales#23 / store_sales#12) AS store_q1_q2_increase#70, (web_sales#68 / web_sales#57) AS web_q2_q3_increase#71, (store_sales#34 / store_sales#23) AS store_q2_q3_increase#72] +Input [10]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, web_sales#57, ca_county#66, web_sales#68] + +(94) Exchange +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(90) Sort [codegen id : 25] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] +(95) Sort [codegen id : 13] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72] Arguments: [ca_county#9 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (94) -+- * ColumnarToRow (93) - +- CometFilter (92) - +- CometScan parquet spark_catalog.default.date_dim (91) +BroadcastExchange (99) ++- * ColumnarToRow (98) + +- CometFilter (97) + +- CometScan parquet spark_catalog.default.date_dim (96) -(91) Scan parquet spark_catalog.default.date_dim +(96) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(92) CometFilter +(97) CometFilter Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) ColumnarToRow [codegen id : 1] +(98) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(94) BroadcastExchange +(99) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (98) -+- * ColumnarToRow (97) - +- CometFilter (96) - +- CometScan parquet spark_catalog.default.date_dim (95) +Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#16 +BroadcastExchange (103) ++- * ColumnarToRow (102) + +- CometFilter (101) + +- CometScan parquet spark_catalog.default.date_dim (100) -(95) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +(100) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(96) CometFilter -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) +(101) CometFilter +Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] +Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) -(97) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +(102) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -(98) BroadcastExchange -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] +(103) BroadcastExchange +Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] -Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 -BroadcastExchange (102) -+- * ColumnarToRow (101) - +- CometFilter (100) - +- CometScan parquet spark_catalog.default.date_dim (99) +Subquery:3 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 +BroadcastExchange (107) ++- * ColumnarToRow (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) -(99) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] +(104) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(100) CometFilter -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) +(105) CometFilter +Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] +Condition : ((((isnotnull(d_qoy#30) AND isnotnull(d_year#29)) AND (d_qoy#30 = 3)) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) -(101) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] +(106) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -(102) BroadcastExchange -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] +(107) BroadcastExchange +Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -Subquery:4 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 59 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#17 +Subquery:5 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#49 IN dynamicpruning#16 -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#29 +Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index f4bf6a89d0..a94a8a94d1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (25) +WholeStageCodegen (13) Sort [ca_county] InputAdapter Exchange [ca_county] #1 - WholeStageCodegen (24) + WholeStageCodegen (12) Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] @@ -14,14 +14,14 @@ WholeStageCodegen (25) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter Exchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,129 +31,120 @@ WholeStageCodegen (25) InputAdapter CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_county] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange #4 + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #5 + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (3) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [ca_county,d_qoy,d_year] #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + CometBroadcastExchange #9 + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [ca_county,d_qoy,d_year] #11 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk] CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 + BroadcastExchange #12 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + CometBroadcastExchange #13 + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #14 + WholeStageCodegen (7) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [ca_county,d_qoy,d_year] #15 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #4 + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) + BroadcastExchange #16 + WholeStageCodegen (9) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #14 - WholeStageCodegen (18) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [ca_county,d_qoy,d_year] #17 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #9 + ReusedExchange [ca_address_sk,ca_county] #5 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) + BroadcastExchange #18 + WholeStageCodegen (11) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - Exchange [ca_county,d_qoy,d_year] #16 - WholeStageCodegen (22) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [ca_county,d_qoy,d_year] #19 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #13 + ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index bd7caaaa11..b7eee60b1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -1,33 +1,35 @@ == Physical Plan == -* HashAggregate (29) -+- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * HashAggregate (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * ColumnarToRow (13) - : : +- CometFilter (12) - : : +- CometScan parquet spark_catalog.default.catalog_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (24) +* HashAggregate (31) ++- Exchange (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * ColumnarToRow (9) + : : +- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.item (3) + : +- BroadcastExchange (23) + : +- * Filter (22) + : +- * HashAggregate (21) + : +- Exchange (20) + : +- * ColumnarToRow (19) + : +- CometHashAggregate (18) + : +- CometProject (17) + : +- CometBroadcastHashJoin (16) + : :- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.catalog_sales (10) + : +- CometBroadcastExchange (15) + : +- CometProject (14) + : +- CometFilter (13) + : +- CometScan parquet spark_catalog.default.date_dim (12) + +- ReusedExchange (26) (1) Scan parquet spark_catalog.default.catalog_sales @@ -42,42 +44,38 @@ ReadSchema: struct Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) -(3) ColumnarToRow [codegen id : 6] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#5, i_manufact_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [i_item_sk#5, i_manufact_id#6] Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) -(6) CometProject +(5) CometProject Input [2]: [i_item_sk#5, i_manufact_id#6] Arguments: [i_item_sk#5], [i_item_sk#5] -(7) ColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [1]: [i_item_sk#5] +Arguments: [i_item_sk#5] -(8) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Right output [1]: [i_item_sk#5] +Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] +(8) CometProject Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] +Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5], [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] + +(9) ColumnarToRow [codegen id : 4] +Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -(11) Scan parquet spark_catalog.default.catalog_sales +(10) Scan parquet spark_catalog.default.catalog_sales Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -85,125 +83,138 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) CometFilter +(11) CometFilter Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] Condition : isnotnull(cs_item_sk#7) -(13) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] +(12) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#11] +(13) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(15) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] -(16) Project [codegen id : 3] -Output [2]: [cs_item_sk#7, cs_ext_discount_amt#8] +(16) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] +Right output [1]: [d_date_sk#11] +Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight + +(17) CometProject Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#11] +Arguments: [cs_item_sk#7, cs_ext_discount_amt#8], [cs_item_sk#7, cs_ext_discount_amt#8] -(17) HashAggregate [codegen id : 3] +(18) CometHashAggregate Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] Keys [1]: [cs_item_sk#7] Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] -Aggregate Attributes [2]: [sum#12, count#13] -Results [3]: [cs_item_sk#7, sum#14, count#15] -(18) Exchange -Input [3]: [cs_item_sk#7, sum#14, count#15] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(19) ColumnarToRow [codegen id : 1] +Input [3]: [cs_item_sk#7, sum#13, count#14] + +(20) Exchange +Input [3]: [cs_item_sk#7, sum#13, count#14] +Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(19) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#7, sum#14, count#15] +(21) HashAggregate [codegen id : 2] +Input [3]: [cs_item_sk#7, sum#13, count#14] Keys [1]: [cs_item_sk#7] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#16] -Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#15] +Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -(20) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) +(22) Filter [codegen id : 2] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) -(21) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] +(23) BroadcastExchange +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=2] -(22) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#5] Right keys [1]: [cs_item_sk#7] Join type: Inner -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#16) -(23) Project [codegen id : 6] +(25) Project [codegen id : 4] Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -(24) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#18] +(26) ReusedExchange [Reuses operator id: 36] +Output [1]: [d_date_sk#17] -(25) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [1]: [cs_ext_discount_amt#2] -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#18] +Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#17] -(27) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 4] Input [1]: [cs_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#19] -Results [1]: [sum#20] +Aggregate Attributes [1]: [sum#18] +Results [1]: [sum#19] -(28) Exchange -Input [1]: [sum#20] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(30) Exchange +Input [1]: [sum#19] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(29) HashAggregate [codegen id : 7] -Input [1]: [sum#20] +(31) HashAggregate [codegen id : 5] +Input [1]: [sum#19] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#21] -Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#21,17,2) AS excess discount amount#22] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#20] +Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#20,17,2) AS excess discount amount#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (34) -+- * ColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.date_dim (30) +BroadcastExchange (36) ++- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(30) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#23] +(32) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#17, d_date#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [d_date_sk#18, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) +(33) CometFilter +Input [2]: [d_date_sk#17, d_date#22] +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#22 <= 2000-04-26)) AND isnotnull(d_date_sk#17)) -(32) CometProject -Input [2]: [d_date_sk#18, d_date#23] -Arguments: [d_date_sk#18], [d_date_sk#18] +(34) CometProject +Input [2]: [d_date_sk#17, d_date#22] +Arguments: [d_date_sk#17], [d_date_sk#17] -(33) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] +(35) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#17] -(34) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) BroadcastExchange +Input [1]: [d_date_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index 146a33fdd2..b8df1e9295 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -1,17 +1,17 @@ -WholeStageCodegen (7) +WholeStageCodegen (5) HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] InputAdapter Exchange #1 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cs_ext_discount_amt] [sum,sum] Project [cs_ext_discount_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_ext_discount_amt,cs_sold_date_sk] BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] - Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk,cs_ext_discount_amt] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -22,31 +22,29 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange #3 + CometProject [i_item_sk] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Filter [(1.3 * avg(cs_ext_discount_amt))] HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] InputAdapter Exchange [cs_item_sk] #5 - WholeStageCodegen (3) - HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] - Project [cs_item_sk,cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [cs_item_sk,cs_ext_discount_amt] + CometProject [cs_item_sk,cs_ext_discount_amt] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index f9541481cd..8e385ed4e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -6,62 +6,62 @@ TakeOrderedAndProject (63) +- Union (59) :- * HashAggregate (28) : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) + : +- * ColumnarToRow (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.customer_address (9) + : +- CometBroadcastExchange (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) : +- CometProject (19) : +- CometFilter (18) : +- CometScan parquet spark_catalog.default.item (17) :- * HashAggregate (43) : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * ColumnarToRow (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) + : +- * ColumnarToRow (41) + : +- CometHashAggregate (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) +- * HashAggregate (58) +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * ColumnarToRow (46) - : : : +- CometFilter (45) - : : : +- CometScan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) + +- * ColumnarToRow (56) + +- CometHashAggregate (55) + +- CometProject (54) + +- CometBroadcastHashJoin (53) + :- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometProject (48) + : : +- CometBroadcastHashJoin (47) + : : :- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet spark_catalog.default.store_sales @@ -76,199 +76,195 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Arguments: [d_date_sk#6], [d_date_sk#6] -(6) Project [codegen id : 5] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] + +(7) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -(7) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] +(9) Scan parquet spark_catalog.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 -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] +(10) CometFilter +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)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] +(11) CometProject +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] +Arguments: [ca_address_sk#9], [ca_address_sk#9] -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [ca_address_sk#9] +Arguments: [ca_address_sk#9] -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +Right output [1]: [ca_address_sk#9] +Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight -(13) Project [codegen id : 5] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] +(14) CometProject +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] +Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] -(14) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_manufact_id#10] +(15) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [i_item_sk#9, i_manufact_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#9, i_manufact_id#10] +(16) CometFilter +Input [2]: [i_item_sk#11, i_manufact_id#12] +Condition : isnotnull(i_item_sk#11) (17) Scan parquet spark_catalog.default.item -Output [2]: [i_category#11, i_manufact_id#12] +Output [2]: [i_category#13, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (18) CometFilter -Input [2]: [i_category#11, i_manufact_id#12] -Condition : (isnotnull(i_category#11) AND (i_category#11 = Electronics )) +Input [2]: [i_category#13, i_manufact_id#14] +Condition : (isnotnull(i_category#13) AND (i_category#13 = Electronics )) (19) CometProject -Input [2]: [i_category#11, i_manufact_id#12] -Arguments: [i_manufact_id#12], [i_manufact_id#12] - -(20) ColumnarToRow [codegen id : 3] -Input [1]: [i_manufact_id#12] - -(21) BroadcastExchange -Input [1]: [i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_manufact_id#10] -Right keys [1]: [i_manufact_id#12] -Join type: LeftSemi -Join condition: None - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_manufact_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#3, i_manufact_id#10] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_manufact_id#10] - -(26) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#3, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_category#13, i_manufact_id#14] +Arguments: [i_manufact_id#14], [i_manufact_id#14] + +(20) CometBroadcastExchange +Input [1]: [i_manufact_id#14] +Arguments: [i_manufact_id#14] + +(21) CometBroadcastHashJoin +Left output [2]: [i_item_sk#11, i_manufact_id#12] +Right output [1]: [i_manufact_id#14] +Arguments: [i_manufact_id#12], [i_manufact_id#14], LeftSemi, BuildRight + +(22) CometBroadcastExchange +Input [2]: [i_item_sk#11, i_manufact_id#12] +Arguments: [i_item_sk#11, i_manufact_id#12] + +(23) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] +Right output [2]: [i_item_sk#11, i_manufact_id#12] +Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight + +(24) CometProject +Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_manufact_id#12] +Arguments: [ss_ext_sales_price#3, i_manufact_id#12], [ss_ext_sales_price#3, i_manufact_id#12] + +(25) CometHashAggregate +Input [2]: [ss_ext_sales_price#3, i_manufact_id#12] +Keys [1]: [i_manufact_id#12] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [i_manufact_id#10, sum#14] + +(26) ColumnarToRow [codegen id : 1] +Input [2]: [i_manufact_id#12, sum#15] (27) Exchange -Input [2]: [i_manufact_id#10, sum#14] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [2]: [i_manufact_id#12, sum#15] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 6] -Input [2]: [i_manufact_id#10, sum#14] -Keys [1]: [i_manufact_id#10] +(28) HashAggregate [codegen id : 2] +Input [2]: [i_manufact_id#12, sum#15] +Keys [1]: [i_manufact_id#12] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] +Results [2]: [i_manufact_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS total_sales#17] (29) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (30) CometFilter -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) - -(31) ColumnarToRow [codegen id : 11] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] - -(32) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#22] - -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] - -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#23] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#23] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] - -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#24, i_manufact_id#25] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#24] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#19, i_manufact_id#25] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_manufact_id#25] - -(41) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#19, i_manufact_id#25] -Keys [1]: [i_manufact_id#25] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_manufact_id#25, sum#27] +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) + +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#23] + +(32) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Right output [1]: [d_date_sk#23] +Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight + +(33) CometProject +Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] +Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] + +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#24] + +(35) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] +Right output [1]: [ca_address_sk#24] +Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight + +(36) CometProject +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] +Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] + +(37) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#25, i_manufact_id#26] + +(38) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] +Right output [2]: [i_item_sk#25, i_manufact_id#26] +Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight + +(39) CometProject +Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_manufact_id#26] +Arguments: [cs_ext_sales_price#20, i_manufact_id#26], [cs_ext_sales_price#20, i_manufact_id#26] + +(40) CometHashAggregate +Input [2]: [cs_ext_sales_price#20, i_manufact_id#26] +Keys [1]: [i_manufact_id#26] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] + +(41) ColumnarToRow [codegen id : 3] +Input [2]: [i_manufact_id#26, sum#27] (42) Exchange -Input [2]: [i_manufact_id#25, sum#27] -Arguments: hashpartitioning(i_manufact_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [2]: [i_manufact_id#26, sum#27] +Arguments: hashpartitioning(i_manufact_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#25, sum#27] -Keys [1]: [i_manufact_id#25] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] -Results [2]: [i_manufact_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] +(43) HashAggregate [codegen id : 4] +Input [2]: [i_manufact_id#26, sum#27] +Keys [1]: [i_manufact_id#26] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#28] +Results [2]: [i_manufact_id#26, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#28,17,2) AS total_sales#29] (44) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] @@ -282,89 +278,84 @@ ReadSchema: struct (65) CometFilter -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 5)) AND isnotnull(d_date_sk#6)) +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) (66) CometProject -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] (67) ColumnarToRow [codegen id : 1] @@ -396,9 +387,9 @@ Input [1]: [d_date_sk#6] (68) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index 4ab82379ff..af2b7cb5d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (20) + WholeStageCodegen (8) HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_manufact_id] #1 - WholeStageCodegen (19) + WholeStageCodegen (7) HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_manufact_id] #2 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_manufact_id] - 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] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_manufact_id,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,75 +31,59 @@ TakeOrderedAndProject [total_sales,i_manufact_id] CometProject [d_date_sk] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_manufact_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (12) + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange #6 + CometBroadcastHashJoin [i_manufact_id,i_manufact_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange #7 + CometProject [i_manufact_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] + WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #7 - WholeStageCodegen (11) - HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_manufact_id] - 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] - ColumnarToRow - InputAdapter + Exchange [i_manufact_id] #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [i_manufact_id,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 - WholeStageCodegen (18) + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 + WholeStageCodegen (6) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #8 - WholeStageCodegen (17) - HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_manufact_id] - 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] - ColumnarToRow - InputAdapter + Exchange [i_manufact_id] #9 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [i_manufact_id,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt index d0f166fe9d..5609dc11c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt @@ -1,36 +1,37 @@ == Physical Plan == -* Sort (32) -+- Exchange (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (24) - : +- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (28) - +- * ColumnarToRow (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.customer (25) +* Sort (33) ++- Exchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Filter (25) + : +- * HashAggregate (24) + : +- Exchange (23) + : +- * ColumnarToRow (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.household_demographics (15) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.customer (26) (1) Scan parquet spark_catalog.default.store_sales @@ -45,174 +46,179 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#7] +(4) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(7) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +(8) CometProject Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#10, s_county#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] +(10) CometFilter +Input [2]: [s_store_sk#10, s_county#11] +Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] +(11) CometProject +Input [2]: [s_store_sk#10, s_county#11] +Arguments: [s_store_sk#10], [s_store_sk#10] -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#10] +Arguments: [s_store_sk#10] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +Right output [1]: [s_store_sk#10] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] +(14) CometProject +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -(14) Scan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +(15) Scan parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) +(16) CometFilter +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Condition : ((((isnotnull(hd_vehicle_count#15) AND ((hd_buy_potential#13 = >10000 ) OR (hd_buy_potential#13 = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN ((cast(hd_dep_count#14 as double) / cast(hd_vehicle_count#15 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#12)) -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] +(17) CometProject +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Arguments: [hd_demo_sk#12], [hd_demo_sk#12] -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#12] +Arguments: [hd_demo_sk#12] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(19) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Right output [1]: [hd_demo_sk#12] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None +(20) CometProject +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] +Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] +(21) CometHashAggregate Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) Exchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) ColumnarToRow [codegen id : 1] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] + +(23) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(23) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +(24) HashAggregate [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] +Aggregate Attributes [1]: [count(1)#17] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(24) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) +(25) Filter [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18] +Condition : ((cnt#18 >= 15) AND (cnt#18 <= 20)) -(25) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(26) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) +(27) CometFilter +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Condition : isnotnull(c_customer_sk#19) -(27) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(28) ColumnarToRow [codegen id : 2] +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(28) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(29) BroadcastExchange +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(29) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#19] Join type: Inner Join condition: None -(30) Project [codegen id : 6] -Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(31) Project [codegen id : 3] +Output [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18, c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(31) Exchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(32) Exchange +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: rangepartitioning(c_last_name#22 ASC NULLS FIRST, c_first_name#21 ASC NULLS FIRST, c_salutation#20 ASC NULLS FIRST, c_preferred_cust_flag#23 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 7] -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST], true, 0 +(33) Sort [codegen id : 4] +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: [c_last_name#22 ASC NULLS FIRST, c_first_name#21 ASC NULLS FIRST, c_salutation#20 ASC NULLS FIRST, c_preferred_cust_flag#23 DESC NULLS LAST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(33) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +(34) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_dom#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] -Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(35) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(35) CometProject -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +(36) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(37) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index 80405a784d..eefd383438 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -1,24 +1,24 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] InputAdapter Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [cnt] HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter Exchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,27 +29,21 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange #5 + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange #6 + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index a46018cdd0..c06c1dd16f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (28) + : : +- * Filter (27) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) + : : : :- * ColumnarToRow (12) + : : : : +- CometBroadcastHashJoin (11) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (10) + : : : : +- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) Scan parquet spark_catalog.default.customer @@ -54,228 +56,237 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#9] +(5) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(6) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(7) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] + +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] +(9) CometProject Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ss_customer_sk#6], [ss_customer_sk#6] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#6] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Right output [1]: [ss_customer_sk#6] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight + +(12) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(13) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#15] -(13) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#13] +(15) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +Right output [1]: [d_date_sk#15] +Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(16) CometProject +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] +Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +(17) ColumnarToRow [codegen id : 1] +Input [1]: [ws_bill_customer_sk#12] -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) BroadcastExchange +Input [1]: [ws_bill_customer_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 9] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] +Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#19] -(20) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#17] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(23) CometProject +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#14] -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] +(24) ColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#16] -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#14] +Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(27) Filter [codegen id : 5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(28) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(27) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +(29) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) +(30) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(29) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] +(31) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#20, ca_state#21] -(30) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(32) BroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(31) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(32) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#19] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#19] +(34) Project [codegen id : 5] +Output [2]: [c_current_cdemo_sk#4, ca_state#21] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(33) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(35) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(34) CometFilter -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Condition : isnotnull(cd_demo_sk#20) +(36) CometFilter +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Condition : isnotnull(cd_demo_sk#22) -(35) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(37) ColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(36) BroadcastExchange -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(38) BroadcastExchange +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#20] +Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 9] -Output [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Input [8]: [c_current_cdemo_sk#4, ca_state#19, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(40) Project [codegen id : 5] +Output [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(39) HashAggregate [codegen id : 9] -Input [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#23), partial_max(cd_dep_count#23), partial_avg(cd_dep_count#23), partial_min(cd_dep_employed_count#24), partial_max(cd_dep_employed_count#24), partial_avg(cd_dep_employed_count#24), partial_min(cd_dep_college_count#25), partial_max(cd_dep_college_count#25), partial_avg(cd_dep_college_count#25)] -Aggregate Attributes [13]: [count#26, min#27, max#28, sum#29, count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38] -Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] +(41) HashAggregate [codegen id : 5] +Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#25), partial_max(cd_dep_count#25), partial_avg(cd_dep_count#25), partial_min(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_avg(cd_dep_employed_count#26), partial_min(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_avg(cd_dep_college_count#27)] +Aggregate Attributes [13]: [count#28, min#29, max#30, sum#31, count#32, min#33, max#34, sum#35, count#36, min#37, max#38, sum#39, count#40] +Results [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53] -(40) Exchange -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(42) Exchange +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53] +Arguments: hashpartitioning(ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(41) HashAggregate [codegen id : 10] -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [count(1), min(cd_dep_count#23), max(cd_dep_count#23), avg(cd_dep_count#23), min(cd_dep_employed_count#24), max(cd_dep_employed_count#24), avg(cd_dep_employed_count#24), min(cd_dep_college_count#25), max(cd_dep_college_count#25), avg(cd_dep_college_count#25)] -Aggregate Attributes [10]: [count(1)#52, min(cd_dep_count#23)#53, max(cd_dep_count#23)#54, avg(cd_dep_count#23)#55, min(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, avg(cd_dep_employed_count#24)#58, min(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, avg(cd_dep_college_count#25)#61] -Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, count(1)#52 AS cnt1#62, min(cd_dep_count#23)#53 AS min(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, avg(cd_dep_count#23)#55 AS avg(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, min(cd_dep_employed_count#24)#56 AS min(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, avg(cd_dep_employed_count#24)#58 AS avg(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, min(cd_dep_college_count#25)#59 AS min(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, avg(cd_dep_college_count#25)#61 AS avg(cd_dep_college_count)#73, cd_dep_count#23] +(43) HashAggregate [codegen id : 6] +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, min#42, max#43, sum#44, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [count(1), min(cd_dep_count#25), max(cd_dep_count#25), avg(cd_dep_count#25), min(cd_dep_employed_count#26), max(cd_dep_employed_count#26), avg(cd_dep_employed_count#26), min(cd_dep_college_count#27), max(cd_dep_college_count#27), avg(cd_dep_college_count#27)] +Aggregate Attributes [10]: [count(1)#54, min(cd_dep_count#25)#55, max(cd_dep_count#25)#56, avg(cd_dep_count#25)#57, min(cd_dep_employed_count#26)#58, max(cd_dep_employed_count#26)#59, avg(cd_dep_employed_count#26)#60, min(cd_dep_college_count#27)#61, max(cd_dep_college_count#27)#62, avg(cd_dep_college_count#27)#63] +Results [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, count(1)#54 AS cnt1#64, min(cd_dep_count#25)#55 AS min(cd_dep_count)#65, max(cd_dep_count#25)#56 AS max(cd_dep_count)#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, cd_dep_employed_count#26, count(1)#54 AS cnt2#68, min(cd_dep_employed_count#26)#58 AS min(cd_dep_employed_count)#69, max(cd_dep_employed_count#26)#59 AS max(cd_dep_employed_count)#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, cd_dep_college_count#27, count(1)#54 AS cnt3#72, min(cd_dep_college_count#27)#61 AS min(cd_dep_college_count)#73, max(cd_dep_college_count#27)#62 AS max(cd_dep_college_count)#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, cd_dep_count#25] -(42) TakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73, cd_dep_count#23] -Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73] +(44) TakeOrderedAndProject +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cnt1#64, min(cd_dep_count)#65, max(cd_dep_count)#66, avg(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, min(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, avg(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, min(cd_dep_college_count)#73, max(cd_dep_college_count)#74, avg(cd_dep_college_count)#75, cd_dep_count#25] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#23, cd_marital_status#24, cnt1#64, min(cd_dep_count)#65, max(cd_dep_count)#66, avg(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, min(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, avg(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, min(cd_dep_college_count)#73, max(cd_dep_college_count)#74, avg(cd_dep_college_count)#75] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(43) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#74, d_qoy#75] +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] -Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) +(46) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(45) CometProject -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +(47) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index ea0ef274ea..efe0b0b4ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (6) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,61 +13,57 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Filter [exists,exists] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt index 39b8381577..33b572ff62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt @@ -1,32 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * Project (27) - +- Window (26) - +- * Sort (25) - +- Exchange (24) - +- * HashAggregate (23) - +- Exchange (22) - +- * HashAggregate (21) - +- * Expand (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store (13) +TakeOrderedAndProject (29) ++- * Project (28) + +- Window (27) + +- * Sort (26) + +- Exchange (25) + +- * HashAggregate (24) + +- Exchange (23) + +- * ColumnarToRow (22) + +- CometHashAggregate (21) + +- CometExpand (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.item (9) + +- CometBroadcastExchange (17) + +- CometProject (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) (1) Scan parquet spark_catalog.default.store_sales @@ -41,154 +42,159 @@ ReadSchema: struct + +(4) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(4) ReusedExchange [Reuses operator id: 33] -Output [1]: [d_date_sk#7] +(5) CometProject +Input [2]: [d_date_sk#7, d_year#8] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +(8) CometProject Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -(7) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] +(9) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#9, i_class#10, i_category#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) +(10) CometFilter +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Condition : isnotnull(i_item_sk#9) -(9) ColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#9, i_category#10] +(11) CometBroadcastExchange +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Arguments: [i_item_sk#9, i_class#10, i_category#11] -(10) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +Right output [3]: [i_item_sk#9, i_class#10, i_category#11] +Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None +(13) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#10, i_category#11] +Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] -(12) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#9, i_category#10] - -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_state#12] +(14) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#12, s_state#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [2]: [s_store_sk#11, s_state#12] -Condition : ((isnotnull(s_state#12) AND (s_state#12 = TN)) AND isnotnull(s_store_sk#11)) - -(15) CometProject -Input [2]: [s_store_sk#11, s_state#12] -Arguments: [s_store_sk#11], [s_store_sk#11] +(15) CometFilter +Input [2]: [s_store_sk#12, s_state#13] +Condition : ((isnotnull(s_state#13) AND (s_state#13 = TN)) AND isnotnull(s_store_sk#12)) -(16) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#11] +(16) CometProject +Input [2]: [s_store_sk#12, s_state#13] +Arguments: [s_store_sk#12], [s_store_sk#12] -(17) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [1]: [s_store_sk#12] +Arguments: [s_store_sk#12] -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] +Right output [1]: [s_store_sk#12] +Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight -(19) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#10, i_class#9] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10, s_store_sk#11] +(19) CometProject +Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11, s_store_sk#12] +Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_category#11, i_class#10], [ss_ext_sales_price#3, ss_net_profit#4, i_category#11, i_class#10] -(20) Expand [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#10, i_class#9] -Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#10, i_class#9, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#10, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#14, spark_grouping_id#15] +(20) CometExpand +Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#11, i_class#10] +Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#11, i_class#10, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#11, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#14, i_class#15, spark_grouping_id#16] -(21) HashAggregate [codegen id : 4] -Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#14, spark_grouping_id#15] -Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15] +(21) CometHashAggregate +Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#14, i_class#15, spark_grouping_id#16] +Keys [3]: [i_category#14, i_class#15, spark_grouping_id#16] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#16, sum#17] -Results [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] -(22) Exchange -Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] -Arguments: hashpartitioning(i_category#13, i_class#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) ColumnarToRow [codegen id : 1] +Input [5]: [i_category#14, i_class#15, spark_grouping_id#16, sum#17, sum#18] + +(23) Exchange +Input [5]: [i_category#14, i_class#15, spark_grouping_id#16, sum#17, sum#18] +Arguments: hashpartitioning(i_category#14, i_class#15, spark_grouping_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(23) HashAggregate [codegen id : 5] -Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] -Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15] +(24) HashAggregate [codegen id : 2] +Input [5]: [i_category#14, i_class#15, spark_grouping_id#16, sum#17, sum#18] +Keys [3]: [i_category#14, i_class#15, spark_grouping_id#16] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledValue(ss_ext_sales_price#3))#21] -Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS _w0#24, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS _w1#25, CASE WHEN (cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint) = 0) THEN i_category#13 END AS _w2#26] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#19, sum(UnscaledValue(ss_ext_sales_price#3))#20] +Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#20,17,2)) AS gross_margin#21, i_category#14, i_class#15, (cast((shiftright(spark_grouping_id#16, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#16, 0) & 1) as tinyint)) AS lochierarchy#22, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#20,17,2)) AS _w0#23, (cast((shiftright(spark_grouping_id#16, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#16, 0) & 1) as tinyint)) AS _w1#24, CASE WHEN (cast((shiftright(spark_grouping_id#16, 0) & 1) as tinyint) = 0) THEN i_category#14 END AS _w2#25] -(24) Exchange -Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -Arguments: hashpartitioning(_w1#25, _w2#26, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(25) Exchange +Input [7]: [gross_margin#21, i_category#14, i_class#15, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(25) Sort [codegen id : 6] -Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -Arguments: [_w1#25 ASC NULLS FIRST, _w2#26 ASC NULLS FIRST, _w0#24 ASC NULLS FIRST], false, 0 +(26) Sort [codegen id : 3] +Input [7]: [gross_margin#21, i_category#14, i_class#15, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST], false, 0 -(26) Window -Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] -Arguments: [rank(_w0#24) windowspecdefinition(_w1#25, _w2#26, _w0#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#27], [_w1#25, _w2#26], [_w0#24 ASC NULLS FIRST] +(27) Window +Input [7]: [gross_margin#21, i_category#14, i_class#15, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] -(27) Project [codegen id : 7] -Output [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] -Input [8]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26, rank_within_parent#27] +(28) Project [codegen id : 4] +Output [5]: [gross_margin#21, i_category#14, i_class#15, lochierarchy#22, rank_within_parent#26] +Input [8]: [gross_margin#21, i_category#14, i_class#15, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] -(28) TakeOrderedAndProject -Input [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] -Arguments: 100, [lochierarchy#23 DESC NULLS LAST, CASE WHEN (lochierarchy#23 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#27 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] +(29) TakeOrderedAndProject +Input [5]: [gross_margin#21, i_category#14, i_class#15, lochierarchy#22, rank_within_parent#26] +Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#14 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#14, i_class#15, lochierarchy#22, rank_within_parent#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) -(29) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#28] +(30) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_year#8] 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 -(30) CometFilter -Input [2]: [d_date_sk#7, d_year#28] -Condition : ((isnotnull(d_year#28) AND (d_year#28 = 2001)) AND isnotnull(d_date_sk#7)) +(31) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(31) CometProject -Input [2]: [d_date_sk#7, d_year#28] +(32) CometProject +Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(32) ColumnarToRow [codegen id : 1] +(33) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(33) BroadcastExchange +(34) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index 7eeb607c3b..574a20fc04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -1,27 +1,27 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [_w1,_w2,_w0] InputAdapter Exchange [_w1,_w2] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] InputAdapter Exchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] - Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -32,20 +32,14 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #5 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange #6 + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt index 23c4ae742b..6eb0f14b1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt @@ -1,29 +1,30 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * ColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan parquet spark_catalog.default.catalog_sales (16) +TakeOrderedAndProject (26) ++- * HashAggregate (25) + +- Exchange (24) + +- * ColumnarToRow (23) + +- CometHashAggregate (22) + +- CometProject (21) + +- CometBroadcastHashJoin (20) + :- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- CometBroadcastExchange (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.inventory (4) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.date_dim (10) + +- CometProject (19) + +- CometFilter (18) + +- CometScan parquet spark_catalog.default.catalog_sales (17) (1) Scan parquet spark_catalog.default.item @@ -41,10 +42,7 @@ Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) A Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(4) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] - -(5) Scan parquet spark_catalog.default.inventory +(4) Scan parquet spark_catalog.default.inventory Output [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -52,128 +50,136 @@ PartitionFilters: [isnotnull(inv_date_sk#8), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Condition : (((isnotnull(inv_quantity_on_hand#7) AND (inv_quantity_on_hand#7 >= 100)) AND (inv_quantity_on_hand#7 <= 500)) AND isnotnull(inv_item_sk#6)) -(7) CometProject +(6) CometProject Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Arguments: [inv_item_sk#6, inv_date_sk#8], [inv_item_sk#6, inv_date_sk#8] -(8) ColumnarToRow [codegen id : 1] -Input [2]: [inv_item_sk#6, inv_date_sk#8] - -(9) BroadcastExchange +(7) CometBroadcastExchange Input [2]: [inv_item_sk#6, inv_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [inv_item_sk#6, inv_date_sk#8] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#6] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Right output [2]: [inv_item_sk#6, inv_date_sk#8] +Arguments: [i_item_sk#1], [inv_item_sk#6], Inner, BuildRight -(11) Project [codegen id : 3] -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] +(9) CometProject Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_item_sk#6, inv_date_sk#8] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] -(12) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#10] +(10) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_date#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] +ReadSchema: struct -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [inv_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(11) CometFilter +Input [2]: [d_date_sk#10, d_date#11] +Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-02-01)) AND (d_date#11 <= 2000-04-01)) AND isnotnull(d_date_sk#10)) -(14) Project [codegen id : 3] -Output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +(12) CometProject +Input [2]: [d_date_sk#10, d_date#11] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] + +(14) CometBroadcastHashJoin +Left output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [inv_date_sk#8], [d_date_sk#10], Inner, BuildRight + +(15) CometProject Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8, d_date_sk#10] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(15) BroadcastExchange +(16) CometBroadcastExchange Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#11, cs_sold_date_sk#12] +(17) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_item_sk#12, cs_sold_date_sk#13] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(17) CometFilter -Input [2]: [cs_item_sk#11, cs_sold_date_sk#12] -Condition : isnotnull(cs_item_sk#11) - -(18) CometProject -Input [2]: [cs_item_sk#11, cs_sold_date_sk#12] -Arguments: [cs_item_sk#11], [cs_item_sk#11] +(18) CometFilter +Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] +Condition : isnotnull(cs_item_sk#12) -(19) ColumnarToRow -Input [1]: [cs_item_sk#11] +(19) CometProject +Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] +Arguments: [cs_item_sk#12], [cs_item_sk#12] -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#11] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Right output [1]: [cs_item_sk#12] +Arguments: [i_item_sk#1], [cs_item_sk#12], Inner, BuildLeft -(21) Project [codegen id : 4] -Output [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, cs_item_sk#11] +(21) CometProject +Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, cs_item_sk#12] +Arguments: [i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_id#2, i_item_desc#3, i_current_price#4] -(22) HashAggregate [codegen id : 4] +(22) CometHashAggregate Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(23) Exchange +(23) ColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) HashAggregate [codegen id : 5] +(24) Exchange +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(25) HashAggregate [codegen id : 2] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] Aggregate Attributes: [] Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(25) TakeOrderedAndProject +(26) TakeOrderedAndProject Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (30) -+- * ColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.date_dim (26) +Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(26) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_date#13] +(27) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_date#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter -Input [2]: [d_date_sk#10, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-01)) AND (d_date#13 <= 2000-04-01)) AND isnotnull(d_date_sk#10)) +(28) CometFilter +Input [2]: [d_date_sk#10, d_date#11] +Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-02-01)) AND (d_date#11 <= 2000-04-01)) AND isnotnull(d_date_sk#10)) -(28) CometProject -Input [2]: [d_date_sk#10, d_date#13] +(29) CometProject +Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(29) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt index 65bb06348b..c3fcd79f0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt @@ -1,44 +1,38 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_current_price] InputAdapter Exchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometProject [i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,cs_item_sk] + CometBroadcastExchange #2 + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometBroadcastExchange #3 + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [cs_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index 5503439ee4..b6b11827d1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -1,51 +1,53 @@ == Physical Plan == -* HashAggregate (47) -+- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin LeftSemi BuildRight (43) - :- * BroadcastHashJoin LeftSemi BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer (7) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- BroadcastExchange (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * ColumnarToRow (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.web_sales (30) - : +- ReusedExchange (33) - +- ReusedExchange (36) +* HashAggregate (49) ++- Exchange (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftSemi BuildRight (45) + :- * BroadcastHashJoin LeftSemi BuildRight (31) + : :- * HashAggregate (17) + : : +- Exchange (16) + : : +- * ColumnarToRow (15) + : : +- CometHashAggregate (14) + : : +- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.customer (9) + : +- BroadcastExchange (30) + : +- * HashAggregate (29) + : +- Exchange (28) + : +- * ColumnarToRow (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (44) + +- * HashAggregate (43) + +- Exchange (42) + +- * ColumnarToRow (41) + +- CometHashAggregate (40) + +- CometProject (39) + +- CometBroadcastHashJoin (38) + :- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometFilter (33) + : : +- CometScan parquet spark_catalog.default.web_sales (32) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.store_sales @@ -60,262 +62,263 @@ ReadSchema: struct Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +(3) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#4, d_date#5] +(4) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#2] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] + +(6) CometBroadcastExchange +Input [2]: [d_date_sk#4, d_date#5] +Arguments: [d_date_sk#4, d_date#5] -(6) Project [codegen id : 3] -Output [2]: [ss_customer_sk#1, d_date#5] +(7) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +Right output [2]: [d_date_sk#4, d_date#5] +Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight + +(8) CometProject Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] +Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] -(7) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +(9) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Condition : isnotnull(c_customer_sk#6) - -(9) ColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +(10) CometFilter +Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Condition : isnotnull(c_customer_sk#7) -(10) BroadcastExchange -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [c_customer_sk#7, c_first_name#8, c_last_name#9] -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#6] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#1, d_date#5] +Right output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight -(12) Project [codegen id : 3] -Output [3]: [c_last_name#8, c_first_name#7, d_date#5] -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#7, c_last_name#8] +(13) CometProject +Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [c_last_name#9, c_first_name#8, d_date#5], [c_last_name#9, c_first_name#8, d_date#5] -(13) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +(14) CometHashAggregate +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] -(14) Exchange -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(15) ColumnarToRow [codegen id : 1] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] + +(16) Exchange +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +(17) HashAggregate [codegen id : 6] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] +Results [3]: [c_last_name#9, c_first_name#8, d_date#5] -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +(18) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(17) CometFilter -Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] -Condition : isnotnull(cs_bill_customer_sk#9) - -(18) ColumnarToRow [codegen id : 6] -Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +(19) CometFilter +Input [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] +Condition : isnotnull(cs_bill_customer_sk#10) -(19) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#12, d_date#13] +(20) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#13, d_date#14] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(21) CometBroadcastHashJoin +Left output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] +Right output [2]: [d_date_sk#13, d_date#14] +Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(21) Project [codegen id : 6] -Output [2]: [cs_bill_customer_sk#9, d_date#13] -Input [4]: [cs_bill_customer_sk#9, cs_sold_date_sk#10, d_date_sk#12, d_date#13] +(22) CometProject +Input [4]: [cs_bill_customer_sk#10, cs_sold_date_sk#11, d_date_sk#13, d_date#14] +Arguments: [cs_bill_customer_sk#10, d_date#14], [cs_bill_customer_sk#10, d_date#14] -(22) ReusedExchange [Reuses operator id: 10] -Output [3]: [c_customer_sk#14, c_first_name#15, c_last_name#16] +(23) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_bill_customer_sk#9] -Right keys [1]: [c_customer_sk#14] -Join type: Inner -Join condition: None +(24) CometBroadcastHashJoin +Left output [2]: [cs_bill_customer_sk#10, d_date#14] +Right output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [cs_bill_customer_sk#10], [c_customer_sk#15], Inner, BuildRight -(24) Project [codegen id : 6] -Output [3]: [c_last_name#16, c_first_name#15, d_date#13] -Input [5]: [cs_bill_customer_sk#9, d_date#13, c_customer_sk#14, c_first_name#15, c_last_name#16] +(25) CometProject +Input [5]: [cs_bill_customer_sk#10, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [c_last_name#17, c_first_name#16, d_date#14], [c_last_name#17, c_first_name#16, d_date#14] -(25) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +(26) CometHashAggregate +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(26) Exchange -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(27) ColumnarToRow [codegen id : 2] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -(27) HashAggregate [codegen id : 7] -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +(28) Exchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(29) HashAggregate [codegen id : 3] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#16, c_first_name#15, d_date#13] +Results [3]: [c_last_name#17, c_first_name#16, d_date#14] -(28) BroadcastExchange -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] +(30) BroadcastExchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] -(29) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#16, ), isnull(c_last_name#16), coalesce(c_first_name#15, ), isnull(c_first_name#15), coalesce(d_date#13, 1970-01-01), isnull(d_date#13)] +(31) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] Join type: LeftSemi Join condition: None -(30) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +(32) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#18), dynamicpruningexpression(ws_sold_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] -Condition : isnotnull(ws_bill_customer_sk#17) - -(32) ColumnarToRow [codegen id : 10] -Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +(33) CometFilter +Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Condition : isnotnull(ws_bill_customer_sk#18) -(33) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#20, d_date#21] +(34) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#21, d_date#22] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(35) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Right output [2]: [d_date_sk#21, d_date#22] +Arguments: [ws_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight -(35) Project [codegen id : 10] -Output [2]: [ws_bill_customer_sk#17, d_date#21] -Input [4]: [ws_bill_customer_sk#17, ws_sold_date_sk#18, d_date_sk#20, d_date#21] +(36) CometProject +Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#21, d_date#22] +Arguments: [ws_bill_customer_sk#18, d_date#22], [ws_bill_customer_sk#18, d_date#22] -(36) ReusedExchange [Reuses operator id: 10] -Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] +(37) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_bill_customer_sk#17] -Right keys [1]: [c_customer_sk#22] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, d_date#22] +Right output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [ws_bill_customer_sk#18], [c_customer_sk#23], Inner, BuildRight -(38) Project [codegen id : 10] -Output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Input [5]: [ws_bill_customer_sk#17, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] +(39) CometProject +Input [5]: [ws_bill_customer_sk#18, d_date#22, c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [c_last_name#25, c_first_name#24, d_date#22], [c_last_name#25, c_first_name#24, d_date#22] -(39) HashAggregate [codegen id : 10] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +(40) CometHashAggregate +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(40) Exchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(41) ColumnarToRow [codegen id : 4] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] + +(42) Exchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: hashpartitioning(c_last_name#25, c_first_name#24, d_date#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(41) HashAggregate [codegen id : 11] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +(43) HashAggregate [codegen id : 5] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] +Results [3]: [c_last_name#25, c_first_name#24, d_date#22] -(42) BroadcastExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] +(44) BroadcastExchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] +(45) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#25, ), isnull(c_last_name#25), coalesce(c_first_name#24, ), isnull(c_first_name#24), coalesce(d_date#22, 1970-01-01), isnull(d_date#22)] Join type: LeftSemi Join condition: None -(44) Project [codegen id : 12] +(46) Project [codegen id : 6] Output: [] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(45) HashAggregate [codegen id : 12] +(47) HashAggregate [codegen id : 6] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#25] -Results [1]: [count#26] +Aggregate Attributes [1]: [count#26] +Results [1]: [count#27] -(46) Exchange -Input [1]: [count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(48) Exchange +Input [1]: [count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(47) HashAggregate [codegen id : 13] -Input [1]: [count#26] +(49) HashAggregate [codegen id : 7] +Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#27] -Results [1]: [count(1)#27 AS count(1)#28] +Aggregate Attributes [1]: [count(1)#28] +Results [1]: [count(1)#28 AS count(1)#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(48) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +(50) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) +(51) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) -(50) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +(52) CometProject +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(52) BroadcastExchange +(54) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 +Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt index 315afe6602..cfac83844a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (13) +WholeStageCodegen (7) HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 - WholeStageCodegen (12) + WholeStageCodegen (6) HashAggregate [count,count] Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] @@ -10,14 +10,14 @@ WholeStageCodegen (13) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometProject [ss_customer_sk,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -28,54 +28,48 @@ WholeStageCodegen (13) CometProject [d_date_sk,d_date] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange #4 + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange #5 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (3) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,d_date] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) + BroadcastExchange #8 + WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [c_last_name,c_first_name,d_date] #9 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometProject [ws_bill_customer_sk,d_date] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index 8a92d8bbc7..6b62725904 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -1,48 +1,52 @@ == Physical Plan == -* Sort (44) -+- Exchange (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (23) - : +- * Filter (22) - : +- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (41) - +- * Project (40) - +- * Filter (39) - +- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * ColumnarToRow (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (27) - : +- ReusedExchange (30) - +- ReusedExchange (33) +* Sort (48) ++- Exchange (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (24) + : +- * Filter (23) + : +- * HashAggregate (22) + : +- Exchange (21) + : +- * ColumnarToRow (20) + : +- CometHashAggregate (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.item (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.warehouse (8) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.date_dim (13) + +- BroadcastExchange (45) + +- * Project (44) + +- * Filter (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * ColumnarToRow (40) + +- CometHashAggregate (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometFilter (26) + : : : +- CometScan parquet spark_catalog.default.inventory (25) + : : +- ReusedExchange (27) + : +- ReusedExchange (30) + +- CometBroadcastExchange (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (1) Scan parquet spark_catalog.default.inventory @@ -57,262 +61,278 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [1]: [i_item_sk#6] Condition : isnotnull(i_item_sk#6) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [1]: [i_item_sk#6] +Arguments: [i_item_sk#6] -(7) BroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [1]: [i_item_sk#6] +Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight -(9) Project [codegen id : 4] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +(7) CometProject Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -(10) Scan parquet spark_catalog.default.warehouse +(8) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Condition : isnotnull(w_warehouse_sk#7) -(12) ColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [w_warehouse_sk#7, w_warehouse_name#8] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#7] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight -(15) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +(12) CometProject Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -(16) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#9, d_moy#10] +(13) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(14) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) + +(15) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] + +(16) CometBroadcastExchange +Input [2]: [d_date_sk#9, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11] -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] +(17) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +Right output [2]: [d_date_sk#9, d_moy#11] +Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +(18) CometProject +Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] +Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] + +(19) CometHashAggregate +Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(20) Exchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(20) ColumnarToRow [codegen id : 1] +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] + +(21) Exchange +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +(22) HashAggregate [codegen id : 4] +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] -Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#17, avg(inv_quantity_on_hand#3)#18] +Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stddev_samp(cast(inv_quantity_on_hand#3 as double))#17 AS stdev#19, avg(inv_quantity_on_hand#3)#18 AS mean#20] -(22) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END +(23) Filter [codegen id : 4] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#19, mean#20] +Condition : CASE WHEN (mean#20 = 0.0) THEN false ELSE ((stdev#19 / mean#20) > 1.0) END -(23) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] +(24) Project [codegen id : 4] +Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#20, CASE WHEN (mean#20 = 0.0) THEN null ELSE (stdev#19 / mean#20) END AS cov#21] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#19, mean#20] -(24) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] +(25) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(inv_date_sk#25), dynamicpruningexpression(inv_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) +(26) CometFilter +Input [4]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25] +Condition : (isnotnull(inv_item_sk#22) AND isnotnull(inv_warehouse_sk#23)) -(26) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] +(27) ReusedExchange [Reuses operator id: 5] +Output [1]: [i_item_sk#27] -(27) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#31] +(28) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25] +Right output [1]: [i_item_sk#27] +Arguments: [inv_item_sk#22], [i_item_sk#27], Inner, BuildRight -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#26] -Right keys [1]: [i_item_sk#31] -Join type: Inner -Join condition: None +(29) CometProject +Input [5]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] +Arguments: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27], [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] -(29) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] +(30) ReusedExchange [Reuses operator id: 10] +Output [2]: [w_warehouse_sk#28, w_warehouse_name#29] -(30) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(31) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] +Right output [2]: [w_warehouse_sk#28, w_warehouse_name#29] +Arguments: [inv_warehouse_sk#23], [w_warehouse_sk#28], Inner, BuildRight -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#27] -Right keys [1]: [w_warehouse_sk#32] -Join type: Inner -Join condition: None +(32) CometProject +Input [6]: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] +Arguments: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29], [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] -(32) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] +(33) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#30, d_year#31, d_moy#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] +ReadSchema: struct -(33) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#34, d_moy#35] +(34) CometFilter +Input [3]: [d_date_sk#30, d_year#31, d_moy#32] +Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(35) CometProject +Input [3]: [d_date_sk#30, d_year#31, d_moy#32] +Arguments: [d_date_sk#30, d_moy#32], [d_date_sk#30, d_moy#32] + +(36) CometBroadcastExchange +Input [2]: [d_date_sk#30, d_moy#32] +Arguments: [d_date_sk#30, d_moy#32] + +(37) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] +Right output [2]: [d_date_sk#30, d_moy#32] +Arguments: [inv_date_sk#25], [d_date_sk#30], Inner, BuildRight + +(38) CometProject +Input [7]: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_date_sk#30, d_moy#32] +Arguments: [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32], [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32] + +(39) CometHashAggregate +Input [5]: [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32] +Keys [4]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#24 as double)), partial_avg(inv_quantity_on_hand#24)] + +(40) ColumnarToRow [codegen id : 2] +Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] + +(41) Exchange +Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] +Arguments: hashpartitioning(w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(42) HashAggregate [codegen id : 3] +Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] +Keys [4]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#24 as double)), avg(inv_quantity_on_hand#24)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#24 as double))#17, avg(inv_quantity_on_hand#24)#18] +Results [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stddev_samp(cast(inv_quantity_on_hand#24 as double))#17 AS stdev#19, avg(inv_quantity_on_hand#24)#18 AS mean#20] + +(43) Filter [codegen id : 3] +Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stdev#19, mean#20] +Condition : CASE WHEN (mean#20 = 0.0) THEN false ELSE ((stdev#19 / mean#20) > 1.0) END + +(44) Project [codegen id : 3] +Output [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#20 AS mean#38, CASE WHEN (mean#20 = 0.0) THEN null ELSE (stdev#19 / mean#20) END AS cov#39] +Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stdev#19, mean#20] + +(45) BroadcastExchange +Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=3] -(35) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] - -(36) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] -Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] -Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(37) Exchange -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(38) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] - -(39) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END - -(40) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] - -(41) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(42) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] +Right keys [2]: [i_item_sk#27, w_warehouse_sk#28] Join type: Inner Join condition: None -(43) Exchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(47) Exchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#20, cov#21, w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#20 ASC NULLS FIRST, cov#21 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#38 ASC NULLS FIRST, cov#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(44) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST], true, 0 +(48) Sort [codegen id : 5] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#20, cov#21, w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] +Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#20 ASC NULLS FIRST, cov#21 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#38 ASC NULLS FIRST, cov#39 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(45) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#48, d_moy#10] +(49) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) +(50) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) -(47) CometProject -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] +(51) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(48) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#10] +(52) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_moy#11] -(49) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(53) BroadcastExchange +Input [2]: [d_date_sk#9, d_moy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#25 IN dynamicpruning#26 +BroadcastExchange (58) ++- * ColumnarToRow (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) -(50) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#49, d_moy#35] +(54) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#30, d_year#31, d_moy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) +(55) CometFilter +Input [3]: [d_date_sk#30, d_year#31, d_moy#32] +Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) -(52) CometProject -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] +(56) CometProject +Input [3]: [d_date_sk#30, d_year#31, d_moy#32] +Arguments: [d_date_sk#30, d_moy#32], [d_date_sk#30, d_moy#32] -(53) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_moy#35] +(57) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#30, d_moy#32] -(54) BroadcastExchange -Input [2]: [d_date_sk#34, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +(58) BroadcastExchange +Input [2]: [d_date_sk#30, d_moy#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 002266e76e..7d28d5c10f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -1,24 +1,24 @@ -WholeStageCodegen (11) +WholeStageCodegen (5) Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] InputAdapter Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) + WholeStageCodegen (4) BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,53 +29,47 @@ WholeStageCodegen (11) CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 + CometBroadcastExchange #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #6 + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) + BroadcastExchange #7 + WholeStageCodegen (3) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange #10 + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index 48ab98a858..85590bf818 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -1,48 +1,52 @@ == Physical Plan == -* Sort (44) -+- Exchange (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (23) - : +- * Filter (22) - : +- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (41) - +- * Project (40) - +- * Filter (39) - +- * HashAggregate (38) - +- Exchange (37) - +- * HashAggregate (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * ColumnarToRow (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (27) - : +- ReusedExchange (30) - +- ReusedExchange (33) +* Sort (48) ++- Exchange (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (24) + : +- * Filter (23) + : +- * HashAggregate (22) + : +- Exchange (21) + : +- * ColumnarToRow (20) + : +- CometHashAggregate (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.item (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.warehouse (8) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.date_dim (13) + +- BroadcastExchange (45) + +- * Project (44) + +- * Filter (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * ColumnarToRow (40) + +- CometHashAggregate (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometFilter (26) + : : : +- CometScan parquet spark_catalog.default.inventory (25) + : : +- ReusedExchange (27) + : +- ReusedExchange (30) + +- CometBroadcastExchange (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (1) Scan parquet spark_catalog.default.inventory @@ -57,262 +61,278 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [1]: [i_item_sk#6] Condition : isnotnull(i_item_sk#6) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [1]: [i_item_sk#6] +Arguments: [i_item_sk#6] -(7) BroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [1]: [i_item_sk#6] +Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight -(9) Project [codegen id : 4] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +(7) CometProject Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -(10) Scan parquet spark_catalog.default.warehouse +(8) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Condition : isnotnull(w_warehouse_sk#7) -(12) ColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [w_warehouse_sk#7, w_warehouse_name#8] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#7] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] +Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight -(15) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +(12) CometProject Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -(16) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#9, d_moy#10] +(13) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(14) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) + +(15) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] + +(16) CometBroadcastExchange +Input [2]: [d_date_sk#9, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11] -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] +(17) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] +Right output [2]: [d_date_sk#9, d_moy#11] +Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +(18) CometProject +Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] +Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] + +(19) CometHashAggregate +Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(20) Exchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(20) ColumnarToRow [codegen id : 1] +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] + +(21) Exchange +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +(22) HashAggregate [codegen id : 4] +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] -Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#17, avg(inv_quantity_on_hand#3)#18] +Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stddev_samp(cast(inv_quantity_on_hand#3 as double))#17 AS stdev#19, avg(inv_quantity_on_hand#3)#18 AS mean#20] -(22) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : (CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END AND CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.5) END) +(23) Filter [codegen id : 4] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#19, mean#20] +Condition : (CASE WHEN (mean#20 = 0.0) THEN false ELSE ((stdev#19 / mean#20) > 1.0) END AND CASE WHEN (mean#20 = 0.0) THEN false ELSE ((stdev#19 / mean#20) > 1.5) END) -(23) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] +(24) Project [codegen id : 4] +Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#20, CASE WHEN (mean#20 = 0.0) THEN null ELSE (stdev#19 / mean#20) END AS cov#21] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#19, mean#20] -(24) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] +(25) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(inv_date_sk#25), dynamicpruningexpression(inv_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) +(26) CometFilter +Input [4]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25] +Condition : (isnotnull(inv_item_sk#22) AND isnotnull(inv_warehouse_sk#23)) -(26) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] +(27) ReusedExchange [Reuses operator id: 5] +Output [1]: [i_item_sk#27] -(27) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#31] +(28) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25] +Right output [1]: [i_item_sk#27] +Arguments: [inv_item_sk#22], [i_item_sk#27], Inner, BuildRight -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#26] -Right keys [1]: [i_item_sk#31] -Join type: Inner -Join condition: None +(29) CometProject +Input [5]: [inv_item_sk#22, inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] +Arguments: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27], [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] -(29) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] +(30) ReusedExchange [Reuses operator id: 10] +Output [2]: [w_warehouse_sk#28, w_warehouse_name#29] -(30) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] +(31) CometBroadcastHashJoin +Left output [4]: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27] +Right output [2]: [w_warehouse_sk#28, w_warehouse_name#29] +Arguments: [inv_warehouse_sk#23], [w_warehouse_sk#28], Inner, BuildRight -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#27] -Right keys [1]: [w_warehouse_sk#32] -Join type: Inner -Join condition: None +(32) CometProject +Input [6]: [inv_warehouse_sk#23, inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] +Arguments: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29], [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] -(32) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] +(33) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#30, d_year#31, d_moy#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] +ReadSchema: struct -(33) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#34, d_moy#35] +(34) CometFilter +Input [3]: [d_date_sk#30, d_year#31, d_moy#32] +Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(35) CometProject +Input [3]: [d_date_sk#30, d_year#31, d_moy#32] +Arguments: [d_date_sk#30, d_moy#32], [d_date_sk#30, d_moy#32] + +(36) CometBroadcastExchange +Input [2]: [d_date_sk#30, d_moy#32] +Arguments: [d_date_sk#30, d_moy#32] + +(37) CometBroadcastHashJoin +Left output [5]: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29] +Right output [2]: [d_date_sk#30, d_moy#32] +Arguments: [inv_date_sk#25], [d_date_sk#30], Inner, BuildRight + +(38) CometProject +Input [7]: [inv_quantity_on_hand#24, inv_date_sk#25, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_date_sk#30, d_moy#32] +Arguments: [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32], [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32] + +(39) CometHashAggregate +Input [5]: [inv_quantity_on_hand#24, i_item_sk#27, w_warehouse_sk#28, w_warehouse_name#29, d_moy#32] +Keys [4]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#24 as double)), partial_avg(inv_quantity_on_hand#24)] + +(40) ColumnarToRow [codegen id : 2] +Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] + +(41) Exchange +Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] +Arguments: hashpartitioning(w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(42) HashAggregate [codegen id : 3] +Input [9]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32, n#33, avg#34, m2#35, sum#36, count#37] +Keys [4]: [w_warehouse_name#29, w_warehouse_sk#28, i_item_sk#27, d_moy#32] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#24 as double)), avg(inv_quantity_on_hand#24)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#24 as double))#17, avg(inv_quantity_on_hand#24)#18] +Results [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stddev_samp(cast(inv_quantity_on_hand#24 as double))#17 AS stdev#19, avg(inv_quantity_on_hand#24)#18 AS mean#20] + +(43) Filter [codegen id : 3] +Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stdev#19, mean#20] +Condition : CASE WHEN (mean#20 = 0.0) THEN false ELSE ((stdev#19 / mean#20) > 1.0) END + +(44) Project [codegen id : 3] +Output [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#20 AS mean#38, CASE WHEN (mean#20 = 0.0) THEN null ELSE (stdev#19 / mean#20) END AS cov#39] +Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, stdev#19, mean#20] + +(45) BroadcastExchange +Input [5]: [w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=3] -(35) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] - -(36) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] -Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] -Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(37) Exchange -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(38) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] - -(39) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END - -(40) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#47] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] - -(41) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(42) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] +Right keys [2]: [i_item_sk#27, w_warehouse_sk#28] Join type: Inner Join condition: None -(43) Exchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(47) Exchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#20, cov#21, w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#20 ASC NULLS FIRST, cov#21 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#38 ASC NULLS FIRST, cov#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(44) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST], true, 0 +(48) Sort [codegen id : 5] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#20, cov#21, w_warehouse_sk#28, i_item_sk#27, d_moy#32, mean#38, cov#39] +Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#20 ASC NULLS FIRST, cov#21 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, mean#38 ASC NULLS FIRST, cov#39 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (49) -+- * ColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(45) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#48, d_moy#10] +(49) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) +(50) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) -(47) CometProject -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] +(51) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(48) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#10] +(52) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_moy#11] -(49) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(53) BroadcastExchange +Input [2]: [d_date_sk#9, d_moy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#25 IN dynamicpruning#26 +BroadcastExchange (58) ++- * ColumnarToRow (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) -(50) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#49, d_moy#35] +(54) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#30, d_year#31, d_moy#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) +(55) CometFilter +Input [3]: [d_date_sk#30, d_year#31, d_moy#32] +Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2001)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) -(52) CometProject -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] +(56) CometProject +Input [3]: [d_date_sk#30, d_year#31, d_moy#32] +Arguments: [d_date_sk#30, d_moy#32], [d_date_sk#30, d_moy#32] -(53) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_moy#35] +(57) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#30, d_moy#32] -(54) BroadcastExchange -Input [2]: [d_date_sk#34, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +(58) BroadcastExchange +Input [2]: [d_date_sk#30, d_moy#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 002266e76e..7d28d5c10f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -1,24 +1,24 @@ -WholeStageCodegen (11) +WholeStageCodegen (5) Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] InputAdapter Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) + WholeStageCodegen (4) BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,53 +29,47 @@ WholeStageCodegen (11) CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 + CometBroadcastExchange #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #6 + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) + BroadcastExchange #7 + WholeStageCodegen (3) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] InputAdapter - Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [i_item_sk] #4 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + CometBroadcastExchange #10 + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index a854a1041b..4aea5922be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -1,112 +1,110 @@ == Physical Plan == -TakeOrderedAndProject (108) -+- * Project (107) - +- * BroadcastHashJoin Inner BuildRight (106) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) +TakeOrderedAndProject (106) ++- * Project (105) + +- * BroadcastHashJoin Inner BuildRight (104) + :- * Project (88) + : +- * BroadcastHashJoin Inner BuildRight (87) : :- * Project (70) : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * Project (52) - : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : :- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Filter (16) - : : : : : +- * HashAggregate (15) - : : : : : +- Exchange (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (10) - : : : : +- BroadcastExchange (32) - : : : : +- * HashAggregate (31) - : : : : +- Exchange (30) - : : : : +- * HashAggregate (29) - : : : : +- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * Project (25) - : : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : : :- * ColumnarToRow (19) - : : : : : : +- CometFilter (18) - : : : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : : : +- BroadcastExchange (23) - : : : : : +- * ColumnarToRow (22) - : : : : : +- CometFilter (21) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : : : +- ReusedExchange (26) - : : : +- BroadcastExchange (50) - : : : +- * Filter (49) - : : : +- * HashAggregate (48) - : : : +- Exchange (47) - : : : +- * HashAggregate (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * ColumnarToRow (36) - : : : : : +- CometFilter (35) - : : : : : +- CometScan parquet spark_catalog.default.customer (34) - : : : : +- BroadcastExchange (40) - : : : : +- * ColumnarToRow (39) - : : : : +- CometFilter (38) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (37) - : : : +- ReusedExchange (43) + : : :- * Project (53) + : : : +- * BroadcastHashJoin Inner BuildRight (52) + : : : :- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Filter (17) + : : : : : +- * HashAggregate (16) + : : : : : +- Exchange (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- CometHashAggregate (13) + : : : : : +- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : : : +- BroadcastExchange (34) + : : : : +- * HashAggregate (33) + : : : : +- Exchange (32) + : : : : +- * ColumnarToRow (31) + : : : : +- CometHashAggregate (30) + : : : : +- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometProject (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (19) + : : : : : : +- CometScan parquet spark_catalog.default.customer (18) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometFilter (21) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : : : +- CometBroadcastExchange (27) + : : : : +- CometFilter (26) + : : : : +- CometScan parquet spark_catalog.default.date_dim (25) + : : : +- BroadcastExchange (51) + : : : +- * Filter (50) + : : : +- * HashAggregate (49) + : : : +- Exchange (48) + : : : +- * ColumnarToRow (47) + : : : +- CometHashAggregate (46) + : : : +- CometProject (45) + : : : +- CometBroadcastHashJoin (44) + : : : :- CometProject (42) + : : : : +- CometBroadcastHashJoin (41) + : : : : :- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.customer (36) + : : : : +- CometBroadcastExchange (40) + : : : : +- CometFilter (39) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (38) + : : : +- ReusedExchange (43) : : +- BroadcastExchange (68) : : +- * HashAggregate (67) : : +- Exchange (66) - : : +- * HashAggregate (65) - : : +- * Project (64) - : : +- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (61) - : : : +- * BroadcastHashJoin Inner BuildRight (60) - : : : :- * ColumnarToRow (55) - : : : : +- CometFilter (54) - : : : : +- CometScan parquet spark_catalog.default.customer (53) - : : : +- BroadcastExchange (59) - : : : +- * ColumnarToRow (58) - : : : +- CometFilter (57) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (56) - : : +- ReusedExchange (62) - : +- BroadcastExchange (87) - : +- * Filter (86) - : +- * HashAggregate (85) - : +- Exchange (84) - : +- * HashAggregate (83) - : +- * Project (82) - : +- * BroadcastHashJoin Inner BuildRight (81) - : :- * Project (79) - : : +- * BroadcastHashJoin Inner BuildRight (78) - : : :- * ColumnarToRow (73) - : : : +- CometFilter (72) - : : : +- CometScan parquet spark_catalog.default.customer (71) - : : +- BroadcastExchange (77) - : : +- * ColumnarToRow (76) - : : +- CometFilter (75) - : : +- CometScan parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (80) - +- BroadcastExchange (105) - +- * HashAggregate (104) - +- Exchange (103) - +- * HashAggregate (102) - +- * Project (101) - +- * BroadcastHashJoin Inner BuildRight (100) - :- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * ColumnarToRow (92) - : : +- CometFilter (91) - : : +- CometScan parquet spark_catalog.default.customer (90) - : +- BroadcastExchange (96) - : +- * ColumnarToRow (95) - : +- CometFilter (94) - : +- CometScan parquet spark_catalog.default.web_sales (93) - +- ReusedExchange (99) + : : +- * ColumnarToRow (65) + : : +- CometHashAggregate (64) + : : +- CometProject (63) + : : +- CometBroadcastHashJoin (62) + : : :- CometProject (60) + : : : +- CometBroadcastHashJoin (59) + : : : :- CometFilter (55) + : : : : +- CometScan parquet spark_catalog.default.customer (54) + : : : +- CometBroadcastExchange (58) + : : : +- CometFilter (57) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (56) + : : +- ReusedExchange (61) + : +- BroadcastExchange (86) + : +- * Filter (85) + : +- * HashAggregate (84) + : +- Exchange (83) + : +- * ColumnarToRow (82) + : +- CometHashAggregate (81) + : +- CometProject (80) + : +- CometBroadcastHashJoin (79) + : :- CometProject (77) + : : +- CometBroadcastHashJoin (76) + : : :- CometFilter (72) + : : : +- CometScan parquet spark_catalog.default.customer (71) + : : +- CometBroadcastExchange (75) + : : +- CometFilter (74) + : : +- CometScan parquet spark_catalog.default.web_sales (73) + : +- ReusedExchange (78) + +- BroadcastExchange (103) + +- * HashAggregate (102) + +- Exchange (101) + +- * ColumnarToRow (100) + +- CometHashAggregate (99) + +- CometProject (98) + +- CometBroadcastHashJoin (97) + :- CometProject (95) + : +- CometBroadcastHashJoin (94) + : :- CometFilter (90) + : : +- CometScan parquet spark_catalog.default.customer (89) + : +- CometBroadcastExchange (93) + : +- CometFilter (92) + : +- CometScan parquet spark_catalog.default.web_sales (91) + +- ReusedExchange (96) (1) Scan parquet spark_catalog.default.customer @@ -120,10 +118,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] Condition : isnotnull(ss_customer_sk#9) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Arguments: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -(7) BroadcastExchange -Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Right output [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Arguments: [c_customer_sk#1], [ss_customer_sk#9], Inner, BuildRight -(9) Project [codegen id : 3] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +(7) CometProject Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -(10) ReusedExchange [Reuses operator id: 112] +(8) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_year#17] +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 -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#14] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(12) Project [codegen id : 3] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] +(10) CometBroadcastExchange +Input [2]: [d_date_sk#16, d_year#17] +Arguments: [d_date_sk#16, d_year#17] + +(11) CometBroadcastHashJoin +Left output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Right output [2]: [d_date_sk#16, d_year#17] +Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight + +(12) CometProject Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14, d_date_sk#16, d_year#17] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] -(13) HashAggregate [codegen id : 3] +(13) CometHashAggregate Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] Functions [1]: [partial_sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] -(14) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(14) ColumnarToRow [codegen id : 1] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#18, isEmpty#19] -(15) HashAggregate [codegen id : 24] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] +(15) Exchange +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#18, isEmpty#19] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(16) HashAggregate [codegen id : 12] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#18, isEmpty#19] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] Functions [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))] -Aggregate Attributes [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22] -Results [2]: [c_customer_id#2 AS customer_id#23, sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22 AS year_total#24] +Aggregate Attributes [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#20] +Results [2]: [c_customer_id#2 AS customer_id#21, sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#20 AS year_total#22] -(16) Filter [codegen id : 24] -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.000000)) +(17) Filter [codegen id : 12] +Input [2]: [customer_id#21, year_total#22] +Condition : (isnotnull(year_total#22) AND (year_total#22 > 0.000000)) -(17) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +(18) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#23, c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_customer_id#26)) - -(19) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +(19) CometFilter +Input [8]: [c_customer_sk#23, c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30] +Condition : (isnotnull(c_customer_sk#23) AND isnotnull(c_customer_id#24)) (20) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Output [6]: [ss_customer_sk#31, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, ss_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (21) CometFilter -Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Condition : isnotnull(ss_customer_sk#33) +Input [6]: [ss_customer_sk#31, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Condition : isnotnull(ss_customer_sk#31) -(22) ColumnarToRow [codegen id : 4] -Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +(22) CometBroadcastExchange +Input [6]: [ss_customer_sk#31, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Arguments: [ss_customer_sk#31, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, ss_sold_date_sk#36] -(23) BroadcastExchange -Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#23, c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30] +Right output [6]: [ss_customer_sk#31, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Arguments: [c_customer_sk#23], [ss_customer_sk#31], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#25] -Right keys [1]: [ss_customer_sk#33] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Input [14]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +(24) CometProject +Input [14]: [c_customer_sk#23, c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, ss_customer_sk#31, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Arguments: [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, ss_sold_date_sk#36] -(26) ReusedExchange [Reuses operator id: 116] -Output [2]: [d_date_sk#40, d_year#41] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] -Input [14]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] - -(29) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] -Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] -Functions [1]: [partial_sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] - -(30) Exchange -Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] -Arguments: hashpartitioning(c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] -Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] -Functions [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] -Aggregate Attributes [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22] -Results [8]: [c_customer_id#26 AS customer_id#46, c_first_name#27 AS customer_first_name#47, c_last_name#28 AS customer_last_name#48, c_preferred_cust_flag#29 AS customer_preferred_cust_flag#49, c_birth_country#30 AS customer_birth_country#50, c_login#31 AS customer_login#51, c_email_address#32 AS customer_email_address#52, sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22 AS year_total#53] - -(32) BroadcastExchange -Input [8]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#38, d_year#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct -(33) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#46] +(26) CometFilter +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) + +(27) CometBroadcastExchange +Input [2]: [d_date_sk#38, d_year#39] +Arguments: [d_date_sk#38, d_year#39] + +(28) CometBroadcastHashJoin +Left output [12]: [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, ss_sold_date_sk#36] +Right output [2]: [d_date_sk#38, d_year#39] +Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight + +(29) CometProject +Input [14]: [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] +Arguments: [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, d_year#39], [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, d_year#39] + +(30) CometHashAggregate +Input [12]: [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, ss_ext_discount_amt#32, ss_ext_sales_price#33, ss_ext_wholesale_cost#34, ss_ext_list_price#35, d_year#39] +Keys [8]: [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, d_year#39] +Functions [1]: [partial_sum(((((ss_ext_list_price#35 - ss_ext_wholesale_cost#34) - ss_ext_discount_amt#32) + ss_ext_sales_price#33) / 2))] + +(31) ColumnarToRow [codegen id : 2] +Input [10]: [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, d_year#39, sum#40, isEmpty#41] + +(32) Exchange +Input [10]: [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, d_year#39, sum#40, isEmpty#41] +Arguments: hashpartitioning(c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, d_year#39, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(33) HashAggregate [codegen id : 3] +Input [10]: [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, d_year#39, sum#40, isEmpty#41] +Keys [8]: [c_customer_id#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_country#28, c_login#29, c_email_address#30, d_year#39] +Functions [1]: [sum(((((ss_ext_list_price#35 - ss_ext_wholesale_cost#34) - ss_ext_discount_amt#32) + ss_ext_sales_price#33) / 2))] +Aggregate Attributes [1]: [sum(((((ss_ext_list_price#35 - ss_ext_wholesale_cost#34) - ss_ext_discount_amt#32) + ss_ext_sales_price#33) / 2))#20] +Results [8]: [c_customer_id#24 AS customer_id#42, c_first_name#25 AS customer_first_name#43, c_last_name#26 AS customer_last_name#44, c_preferred_cust_flag#27 AS customer_preferred_cust_flag#45, c_birth_country#28 AS customer_birth_country#46, c_login#29 AS customer_login#47, c_email_address#30 AS customer_email_address#48, sum(((((ss_ext_list_price#35 - ss_ext_wholesale_cost#34) - ss_ext_discount_amt#32) + ss_ext_sales_price#33) / 2))#20 AS year_total#49] + +(34) BroadcastExchange +Input [8]: [customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(35) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [customer_id#21] +Right keys [1]: [customer_id#42] Join type: Inner Join condition: None -(34) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] +(36) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter -Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] -Condition : (isnotnull(c_customer_sk#54) AND isnotnull(c_customer_id#55)) +(37) CometFilter +Input [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] +Condition : (isnotnull(c_customer_sk#50) AND isnotnull(c_customer_id#51)) -(36) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] - -(37) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +(38) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_bill_customer_sk#58, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#67), dynamicpruningexpression(cs_sold_date_sk#67 IN dynamicpruning#68)] +PartitionFilters: [isnotnull(cs_sold_date_sk#63), dynamicpruningexpression(cs_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -Condition : isnotnull(cs_bill_customer_sk#62) +(39) CometFilter +Input [6]: [cs_bill_customer_sk#58, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63] +Condition : isnotnull(cs_bill_customer_sk#58) -(39) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +(40) CometBroadcastExchange +Input [6]: [cs_bill_customer_sk#58, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63] +Arguments: [cs_bill_customer_sk#58, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63] -(40) BroadcastExchange -Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(41) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] +Right output [6]: [cs_bill_customer_sk#58, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63] +Arguments: [c_customer_sk#50], [cs_bill_customer_sk#58], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#54] -Right keys [1]: [cs_bill_customer_sk#62] -Join type: Inner -Join condition: None +(42) CometProject +Input [14]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_bill_customer_sk#58, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63] +Arguments: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63], [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63] -(42) Project [codegen id : 10] -Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] -Input [14]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +(43) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#65, d_year#66] -(43) ReusedExchange [Reuses operator id: 112] -Output [2]: [d_date_sk#69, d_year#70] +(44) CometBroadcastHashJoin +Left output [12]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63] +Right output [2]: [d_date_sk#65, d_year#66] +Arguments: [cs_sold_date_sk#63], [d_date_sk#65], Inner, BuildRight -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#67] -Right keys [1]: [d_date_sk#69] -Join type: Inner -Join condition: None +(45) CometProject +Input [14]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, cs_sold_date_sk#63, d_date_sk#65, d_year#66] +Arguments: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, d_year#66], [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, d_year#66] + +(46) CometHashAggregate +Input [12]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, cs_ext_discount_amt#59, cs_ext_sales_price#60, cs_ext_wholesale_cost#61, cs_ext_list_price#62, d_year#66] +Keys [8]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66] +Functions [1]: [partial_sum(((((cs_ext_list_price#62 - cs_ext_wholesale_cost#61) - cs_ext_discount_amt#59) + cs_ext_sales_price#60) / 2))] -(45) Project [codegen id : 10] -Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] -Input [14]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67, d_date_sk#69, d_year#70] - -(46) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] -Functions [1]: [partial_sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] -Aggregate Attributes [2]: [sum#71, isEmpty#72] -Results [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] - -(47) Exchange -Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(48) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] -Functions [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75] -Results [2]: [c_customer_id#55 AS customer_id#76, sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75 AS year_total#77] - -(49) Filter [codegen id : 11] -Input [2]: [customer_id#76, year_total#77] -Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.000000)) - -(50) BroadcastExchange -Input [2]: [customer_id#76, year_total#77] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(51) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#76] +(47) ColumnarToRow [codegen id : 4] +Input [10]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66, sum#67, isEmpty#68] + +(48) Exchange +Input [10]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66, sum#67, isEmpty#68] +Arguments: hashpartitioning(c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(49) HashAggregate [codegen id : 5] +Input [10]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66, sum#67, isEmpty#68] +Keys [8]: [c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57, d_year#66] +Functions [1]: [sum(((((cs_ext_list_price#62 - cs_ext_wholesale_cost#61) - cs_ext_discount_amt#59) + cs_ext_sales_price#60) / 2))] +Aggregate Attributes [1]: [sum(((((cs_ext_list_price#62 - cs_ext_wholesale_cost#61) - cs_ext_discount_amt#59) + cs_ext_sales_price#60) / 2))#69] +Results [2]: [c_customer_id#51 AS customer_id#70, sum(((((cs_ext_list_price#62 - cs_ext_wholesale_cost#61) - cs_ext_discount_amt#59) + cs_ext_sales_price#60) / 2))#69 AS year_total#71] + +(50) Filter [codegen id : 5] +Input [2]: [customer_id#70, year_total#71] +Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.000000)) + +(51) BroadcastExchange +Input [2]: [customer_id#70, year_total#71] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(52) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [customer_id#21] +Right keys [1]: [customer_id#70] Join type: Inner Join condition: None -(52) Project [codegen id : 24] -Output [11]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77] -Input [12]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, customer_id#76, year_total#77] +(53) Project [codegen id : 12] +Output [11]: [customer_id#21, year_total#22, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#49, year_total#71] +Input [12]: [customer_id#21, year_total#22, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#49, customer_id#70, year_total#71] -(53) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] +(54) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] -Condition : (isnotnull(c_customer_sk#78) AND isnotnull(c_customer_id#79)) - -(55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] +(55) CometFilter +Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] +Condition : (isnotnull(c_customer_sk#72) AND isnotnull(c_customer_id#73)) (56) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +Output [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(cs_sold_date_sk#85), dynamicpruningexpression(cs_sold_date_sk#85 IN dynamicpruning#86)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (57) CometFilter -Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] -Condition : isnotnull(cs_bill_customer_sk#86) +Input [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] +Condition : isnotnull(cs_bill_customer_sk#80) -(58) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +(58) CometBroadcastExchange +Input [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] +Arguments: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] -(59) BroadcastExchange -Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(59) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] +Right output [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] +Arguments: [c_customer_sk#72], [cs_bill_customer_sk#80], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#78] -Right keys [1]: [cs_bill_customer_sk#86] -Join type: Inner -Join condition: None +(60) CometProject +Input [14]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] +Arguments: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85], [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] -(61) Project [codegen id : 14] -Output [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] -Input [14]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +(61) ReusedExchange [Reuses operator id: 27] +Output [2]: [d_date_sk#87, d_year#88] -(62) ReusedExchange [Reuses operator id: 116] -Output [2]: [d_date_sk#93, d_year#94] +(62) CometBroadcastHashJoin +Left output [12]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] +Right output [2]: [d_date_sk#87, d_year#88] +Arguments: [cs_sold_date_sk#85], [d_date_sk#87], Inner, BuildRight -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#91] -Right keys [1]: [d_date_sk#93] -Join type: Inner -Join condition: None +(63) CometProject +Input [14]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85, d_date_sk#87, d_year#88] +Arguments: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, d_year#88], [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, d_year#88] -(64) Project [codegen id : 14] -Output [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] -Input [14]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91, d_date_sk#93, d_year#94] +(64) CometHashAggregate +Input [12]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, d_year#88] +Keys [8]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88] +Functions [1]: [partial_sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))] -(65) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] -Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94] -Functions [1]: [partial_sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))] -Aggregate Attributes [2]: [sum#95, isEmpty#96] -Results [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] +(65) ColumnarToRow [codegen id : 6] +Input [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88, sum#89, isEmpty#90] (66) Exchange -Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] -Arguments: hashpartitioning(c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88, sum#89, isEmpty#90] +Arguments: hashpartitioning(c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(67) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] -Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94] -Functions [1]: [sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))#75] -Results [2]: [c_customer_id#79 AS customer_id#99, sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))#75 AS year_total#100] +(67) HashAggregate [codegen id : 7] +Input [10]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88, sum#89, isEmpty#90] +Keys [8]: [c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79, d_year#88] +Functions [1]: [sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))] +Aggregate Attributes [1]: [sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))#69] +Results [2]: [c_customer_id#73 AS customer_id#91, sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))#69 AS year_total#92] (68) BroadcastExchange -Input [2]: [customer_id#99, year_total#100] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +Input [2]: [customer_id#91, year_total#92] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(69) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#99] +(69) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [customer_id#21] +Right keys [1]: [customer_id#91] Join type: Inner -Join condition: (CASE WHEN (year_total#77 > 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END) +Join condition: (CASE WHEN (year_total#71 > 0.000000) THEN (year_total#92 / year_total#71) END > CASE WHEN (year_total#22 > 0.000000) THEN (year_total#49 / year_total#22) END) -(70) Project [codegen id : 24] -Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100] -Input [13]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77, customer_id#99, year_total#100] +(70) Project [codegen id : 12] +Output [10]: [customer_id#21, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#71, year_total#92] +Input [13]: [customer_id#21, year_total#22, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#49, year_total#71, customer_id#91, year_total#92] (71) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Output [8]: [c_customer_sk#93, c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (72) CometFilter -Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] -Condition : (isnotnull(c_customer_sk#101) AND isnotnull(c_customer_id#102)) - -(73) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Input [8]: [c_customer_sk#93, c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100] +Condition : (isnotnull(c_customer_sk#93) AND isnotnull(c_customer_id#94)) -(74) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +(73) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#114), dynamicpruningexpression(ws_sold_date_sk#114 IN dynamicpruning#115)] +PartitionFilters: [isnotnull(ws_sold_date_sk#106), dynamicpruningexpression(ws_sold_date_sk#106 IN dynamicpruning#107)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(75) CometFilter -Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] -Condition : isnotnull(ws_bill_customer_sk#109) - -(76) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] - -(77) BroadcastExchange -Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#101] -Right keys [1]: [ws_bill_customer_sk#109] +(74) CometFilter +Input [6]: [ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] +Condition : isnotnull(ws_bill_customer_sk#101) + +(75) CometBroadcastExchange +Input [6]: [ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] +Arguments: [ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] + +(76) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#93, c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100] +Right output [6]: [ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] +Arguments: [c_customer_sk#93], [ws_bill_customer_sk#101], Inner, BuildRight + +(77) CometProject +Input [14]: [c_customer_sk#93, c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_bill_customer_sk#101, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] +Arguments: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106], [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] + +(78) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#108, d_year#109] + +(79) CometBroadcastHashJoin +Left output [12]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106] +Right output [2]: [d_date_sk#108, d_year#109] +Arguments: [ws_sold_date_sk#106], [d_date_sk#108], Inner, BuildRight + +(80) CometProject +Input [14]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, ws_sold_date_sk#106, d_date_sk#108, d_year#109] +Arguments: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, d_year#109], [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, d_year#109] + +(81) CometHashAggregate +Input [12]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, ws_ext_discount_amt#102, ws_ext_sales_price#103, ws_ext_wholesale_cost#104, ws_ext_list_price#105, d_year#109] +Keys [8]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109] +Functions [1]: [partial_sum(((((ws_ext_list_price#105 - ws_ext_wholesale_cost#104) - ws_ext_discount_amt#102) + ws_ext_sales_price#103) / 2))] + +(82) ColumnarToRow [codegen id : 8] +Input [10]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109, sum#110, isEmpty#111] + +(83) Exchange +Input [10]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109, sum#110, isEmpty#111] +Arguments: hashpartitioning(c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(84) HashAggregate [codegen id : 9] +Input [10]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109, sum#110, isEmpty#111] +Keys [8]: [c_customer_id#94, c_first_name#95, c_last_name#96, c_preferred_cust_flag#97, c_birth_country#98, c_login#99, c_email_address#100, d_year#109] +Functions [1]: [sum(((((ws_ext_list_price#105 - ws_ext_wholesale_cost#104) - ws_ext_discount_amt#102) + ws_ext_sales_price#103) / 2))] +Aggregate Attributes [1]: [sum(((((ws_ext_list_price#105 - ws_ext_wholesale_cost#104) - ws_ext_discount_amt#102) + ws_ext_sales_price#103) / 2))#112] +Results [2]: [c_customer_id#94 AS customer_id#113, sum(((((ws_ext_list_price#105 - ws_ext_wholesale_cost#104) - ws_ext_discount_amt#102) + ws_ext_sales_price#103) / 2))#112 AS year_total#114] + +(85) Filter [codegen id : 9] +Input [2]: [customer_id#113, year_total#114] +Condition : (isnotnull(year_total#114) AND (year_total#114 > 0.000000)) + +(86) BroadcastExchange +Input [2]: [customer_id#113, year_total#114] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=9] + +(87) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [customer_id#21] +Right keys [1]: [customer_id#113] Join type: Inner Join condition: None -(79) Project [codegen id : 18] -Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] -Input [14]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +(88) Project [codegen id : 12] +Output [11]: [customer_id#21, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#71, year_total#92, year_total#114] +Input [12]: [customer_id#21, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#71, year_total#92, customer_id#113, year_total#114] -(80) ReusedExchange [Reuses operator id: 112] -Output [2]: [d_date_sk#116, d_year#117] - -(81) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#114] -Right keys [1]: [d_date_sk#116] -Join type: Inner -Join condition: None - -(82) Project [codegen id : 18] -Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] -Input [14]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114, d_date_sk#116, d_year#117] - -(83) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] -Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] -Functions [1]: [partial_sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] -Aggregate Attributes [2]: [sum#118, isEmpty#119] -Results [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] - -(84) Exchange -Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] -Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(85) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] -Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] -Functions [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122] -Results [2]: [c_customer_id#102 AS customer_id#123, sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122 AS year_total#124] - -(86) Filter [codegen id : 19] -Input [2]: [customer_id#123, year_total#124] -Condition : (isnotnull(year_total#124) AND (year_total#124 > 0.000000)) - -(87) BroadcastExchange -Input [2]: [customer_id#123, year_total#124] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] - -(88) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#123] -Join type: Inner -Join condition: None - -(89) Project [codegen id : 24] -Output [11]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124] -Input [12]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, customer_id#123, year_total#124] - -(90) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +(89) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(91) CometFilter -Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] -Condition : (isnotnull(c_customer_sk#125) AND isnotnull(c_customer_id#126)) - -(92) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +(90) CometFilter +Input [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] +Condition : (isnotnull(c_customer_sk#115) AND isnotnull(c_customer_id#116)) -(93) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +(91) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#138), dynamicpruningexpression(ws_sold_date_sk#138 IN dynamicpruning#139)] +PartitionFilters: [isnotnull(ws_sold_date_sk#128), dynamicpruningexpression(ws_sold_date_sk#128 IN dynamicpruning#129)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(94) CometFilter -Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] -Condition : isnotnull(ws_bill_customer_sk#133) +(92) CometFilter +Input [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] +Condition : isnotnull(ws_bill_customer_sk#123) -(95) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +(93) CometBroadcastExchange +Input [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] +Arguments: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] -(96) BroadcastExchange -Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] +(94) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] +Right output [6]: [ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] +Arguments: [c_customer_sk#115], [ws_bill_customer_sk#123], Inner, BuildRight -(97) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#125] -Right keys [1]: [ws_bill_customer_sk#133] -Join type: Inner -Join condition: None +(95) CometProject +Input [14]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_bill_customer_sk#123, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] +Arguments: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128], [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] -(98) Project [codegen id : 22] -Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] -Input [14]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +(96) ReusedExchange [Reuses operator id: 27] +Output [2]: [d_date_sk#130, d_year#131] -(99) ReusedExchange [Reuses operator id: 116] -Output [2]: [d_date_sk#140, d_year#141] +(97) CometBroadcastHashJoin +Left output [12]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128] +Right output [2]: [d_date_sk#130, d_year#131] +Arguments: [ws_sold_date_sk#128], [d_date_sk#130], Inner, BuildRight -(100) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#138] -Right keys [1]: [d_date_sk#140] -Join type: Inner -Join condition: None +(98) CometProject +Input [14]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, ws_sold_date_sk#128, d_date_sk#130, d_year#131] +Arguments: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, d_year#131], [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, d_year#131] + +(99) CometHashAggregate +Input [12]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, ws_ext_discount_amt#124, ws_ext_sales_price#125, ws_ext_wholesale_cost#126, ws_ext_list_price#127, d_year#131] +Keys [8]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131] +Functions [1]: [partial_sum(((((ws_ext_list_price#127 - ws_ext_wholesale_cost#126) - ws_ext_discount_amt#124) + ws_ext_sales_price#125) / 2))] + +(100) ColumnarToRow [codegen id : 10] +Input [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#132, isEmpty#133] + +(101) Exchange +Input [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#132, isEmpty#133] +Arguments: hashpartitioning(c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(101) Project [codegen id : 22] -Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] -Input [14]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138, d_date_sk#140, d_year#141] - -(102) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] -Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] -Functions [1]: [partial_sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] -Aggregate Attributes [2]: [sum#142, isEmpty#143] -Results [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] - -(103) Exchange -Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] -Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, 5), ENSURE_REQUIREMENTS, [plan_id=16] - -(104) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] -Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] -Functions [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122] -Results [2]: [c_customer_id#126 AS customer_id#146, sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122 AS year_total#147] - -(105) BroadcastExchange -Input [2]: [customer_id#146, year_total#147] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] - -(106) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#146] +(102) HashAggregate [codegen id : 11] +Input [10]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131, sum#132, isEmpty#133] +Keys [8]: [c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122, d_year#131] +Functions [1]: [sum(((((ws_ext_list_price#127 - ws_ext_wholesale_cost#126) - ws_ext_discount_amt#124) + ws_ext_sales_price#125) / 2))] +Aggregate Attributes [1]: [sum(((((ws_ext_list_price#127 - ws_ext_wholesale_cost#126) - ws_ext_discount_amt#124) + ws_ext_sales_price#125) / 2))#112] +Results [2]: [c_customer_id#116 AS customer_id#134, sum(((((ws_ext_list_price#127 - ws_ext_wholesale_cost#126) - ws_ext_discount_amt#124) + ws_ext_sales_price#125) / 2))#112 AS year_total#135] + +(103) BroadcastExchange +Input [2]: [customer_id#134, year_total#135] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] + +(104) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [customer_id#21] +Right keys [1]: [customer_id#134] Join type: Inner -Join condition: (CASE WHEN (year_total#77 > 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#124 > 0.000000) THEN (year_total#147 / year_total#124) END) +Join condition: (CASE WHEN (year_total#71 > 0.000000) THEN (year_total#92 / year_total#71) END > CASE WHEN (year_total#114 > 0.000000) THEN (year_total#135 / year_total#114) END) -(107) Project [codegen id : 24] -Output [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] -Input [13]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124, customer_id#146, year_total#147] +(105) Project [codegen id : 12] +Output [7]: [customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48] +Input [13]: [customer_id#21, customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48, year_total#71, year_total#92, year_total#114, customer_id#134, year_total#135] -(108) TakeOrderedAndProject -Input [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] -Arguments: 100, [customer_id#46 ASC NULLS FIRST, customer_first_name#47 ASC NULLS FIRST, customer_last_name#48 ASC NULLS FIRST, customer_preferred_cust_flag#49 ASC NULLS FIRST, customer_birth_country#50 ASC NULLS FIRST, customer_login#51 ASC NULLS FIRST, customer_email_address#52 ASC NULLS FIRST], [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] +(106) TakeOrderedAndProject +Input [7]: [customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48] +Arguments: 100, [customer_id#42 ASC NULLS FIRST, customer_first_name#43 ASC NULLS FIRST, customer_last_name#44 ASC NULLS FIRST, customer_preferred_cust_flag#45 ASC NULLS FIRST, customer_birth_country#46 ASC NULLS FIRST, customer_login#47 ASC NULLS FIRST, customer_email_address#48 ASC NULLS FIRST], [customer_id#42, customer_first_name#43, customer_last_name#44, customer_preferred_cust_flag#45, customer_birth_country#46, customer_login#47, customer_email_address#48] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 -BroadcastExchange (112) -+- * ColumnarToRow (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 +BroadcastExchange (110) ++- * ColumnarToRow (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(109) Scan parquet spark_catalog.default.date_dim +(107) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#16, d_year#17] 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 -(110) CometFilter +(108) CometFilter Input [2]: [d_date_sk#16, d_year#17] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(111) ColumnarToRow [codegen id : 1] +(109) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] -(112) BroadcastExchange +(110) BroadcastExchange Input [2]: [d_date_sk#16, d_year#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 -BroadcastExchange (116) -+- * ColumnarToRow (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 +BroadcastExchange (114) ++- * ColumnarToRow (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) -(113) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_year#41] +(111) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#38, d_year#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter -Input [2]: [d_date_sk#40, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) +(112) CometFilter +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(115) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#40, d_year#41] +(113) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#38, d_year#39] -(116) BroadcastExchange -Input [2]: [d_date_sk#40, d_year#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] +(114) BroadcastExchange +Input [2]: [d_date_sk#38, d_year#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 38 Hosting Expression = cs_sold_date_sk#63 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#39 +Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#85 IN dynamicpruning#37 -Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#114 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 73 Hosting Expression = ws_sold_date_sk#106 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#39 +Subquery:6 Hosting operator id = 91 Hosting Expression = ws_sold_date_sk#128 IN dynamicpruning#37 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt index 99e255a0e4..048da153be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - WholeStageCodegen (24) + WholeStageCodegen (12) Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] @@ -13,167 +13,141 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange #7 CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #12 + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #13 + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [cs_bill_customer_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #15 + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 InputAdapter - BroadcastExchange #14 - WholeStageCodegen (19) + BroadcastExchange #16 + WholeStageCodegen (9) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 - WholeStageCodegen (18) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #18 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #17 - WholeStageCodegen (23) + BroadcastExchange #19 + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 - WholeStageCodegen (22) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #21 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt index d51d63d8a5..565b30e9a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt @@ -1,25 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (19) ++- * HashAggregate (18) + +- Exchange (17) + +- * ColumnarToRow (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- CometBroadcastExchange (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.store_sales (4) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.item (9) (1) Scan parquet spark_catalog.default.date_dim @@ -37,10 +35,7 @@ Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) Input [3]: [d_date_sk#1, d_year#2, d_moy#3] Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] -(4) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#1, d_year#2] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -48,78 +43,71 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Condition : isnotnull(ss_item_sk#4) -(7) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [d_date_sk#1, d_year#2] +Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight -(10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +(8) CometProject Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -(11) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] 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 -(12) CometFilter +(10) CometFilter Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) -(13) CometProject +(11) CometProject Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] Arguments: [i_item_sk#7, i_category_id#8, i_category#9], [i_item_sk#7, i_category_id#8, i_category#9] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [3]: [i_item_sk#7, i_category_id#8, i_category#9] +Arguments: [i_item_sk#7, i_category_id#8, i_category#9] -(15) BroadcastExchange -Input [3]: [i_item_sk#7, i_category_id#8, i_category#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#4] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +Right output [3]: [i_item_sk#7, i_category_id#8, i_category#9] +Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight -(17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] +(14) CometProject Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#9] +Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] -(18) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] Keys [3]: [d_year#2, i_category_id#8, i_category#9] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#11] -Results [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] -(19) Exchange -Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(16) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#11] + +(17) Exchange +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#11] +Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] -Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] +(18) HashAggregate [codegen id : 2] +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#11] Keys [3]: [d_year#2, i_category_id#8, i_category#9] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_category_id#8, i_category#9, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum(ss_ext_sales_price)#14] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#12] +Results [4]: [d_year#2, i_category_id#8, i_category#9, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#12,17,2) AS sum(ss_ext_sales_price)#13] -(21) TakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] -Arguments: 100, [sum(ss_ext_sales_price)#14 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] +(19) TakeOrderedAndProject +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#13] +Arguments: 100, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt index 67906b8c7a..3e69a33417 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt @@ -1,31 +1,23 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] InputAdapter Exchange [d_year,i_category_id,i_category] #1 - WholeStageCodegen (3) - HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] - Project [d_year,ss_ext_sales_price,i_category_id,i_category] - 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] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] + CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_year] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_category_id,i_category] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_category_id,i_category] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt index e892aa4694..a2cfe78982 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt @@ -1,25 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.store (11) +TakeOrderedAndProject (19) ++- * HashAggregate (18) + +- Exchange (17) + +- * ColumnarToRow (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- CometBroadcastExchange (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.store_sales (4) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.store (9) (1) Scan parquet spark_catalog.default.date_dim @@ -37,10 +35,7 @@ Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] Arguments: [d_date_sk#1, d_day_name#3], [d_date_sk#1, d_day_name#3] -(4) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#1, d_day_name#3] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -48,78 +43,71 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Condition : isnotnull(ss_store_sk#4) -(7) ColumnarToRow [codegen id : 1] -Input [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [d_date_sk#1, d_day_name#3] +Right output [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight -(10) Project [codegen id : 3] -Output [3]: [d_day_name#3, ss_store_sk#4, ss_sales_price#5] +(8) CometProject Input [5]: [d_date_sk#1, d_day_name#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_day_name#3, ss_store_sk#4, ss_sales_price#5], [d_day_name#3, ss_store_sk#4, ss_sales_price#5] -(11) Scan parquet spark_catalog.default.store +(9) Scan parquet spark_catalog.default.store Output [4]: [s_store_sk#7, s_store_id#8, s_store_name#9, s_gmt_offset#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [4]: [s_store_sk#7, s_store_id#8, s_store_name#9, s_gmt_offset#10] Condition : ((isnotnull(s_gmt_offset#10) AND (s_gmt_offset#10 = -5.00)) AND isnotnull(s_store_sk#7)) -(13) CometProject +(11) CometProject Input [4]: [s_store_sk#7, s_store_id#8, s_store_name#9, s_gmt_offset#10] Arguments: [s_store_sk#7, s_store_id#8, s_store_name#9], [s_store_sk#7, s_store_id#8, s_store_name#9] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [3]: [s_store_sk#7, s_store_id#8, s_store_name#9] +Arguments: [s_store_sk#7, s_store_id#8, s_store_name#9] -(15) BroadcastExchange -Input [3]: [s_store_sk#7, s_store_id#8, s_store_name#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [d_day_name#3, ss_store_sk#4, ss_sales_price#5] +Right output [3]: [s_store_sk#7, s_store_id#8, s_store_name#9] +Arguments: [ss_store_sk#4], [s_store_sk#7], Inner, BuildRight -(17) Project [codegen id : 3] -Output [4]: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] +(14) CometProject Input [6]: [d_day_name#3, ss_store_sk#4, ss_sales_price#5, s_store_sk#7, s_store_id#8, s_store_name#9] +Arguments: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9], [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] -(18) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [4]: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] Keys [2]: [s_store_name#9, s_store_id#8] Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))] -Aggregate Attributes [7]: [sum#11, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17] -Results [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] -(19) Exchange -Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(s_store_name#9, s_store_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(16) ColumnarToRow [codegen id : 1] +Input [9]: [s_store_name#9, s_store_id#8, sum#11, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17] + +(17) Exchange +Input [9]: [s_store_name#9, s_store_id#8, sum#11, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17] +Arguments: hashpartitioning(s_store_name#9, s_store_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] -Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] +(18) HashAggregate [codegen id : 2] +Input [9]: [s_store_name#9, s_store_id#8, sum#11, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17] Keys [2]: [s_store_name#9, s_store_id#8] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31] -Results [9]: [s_store_name#9, s_store_id#8, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25,17,2) AS sun_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26,17,2) AS mon_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27,17,2) AS tue_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28,17,2) AS wed_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29,17,2) AS thu_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30,17,2) AS fri_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31,17,2) AS sat_sales#38] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#18, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#24] +Results [9]: [s_store_name#9, s_store_id#8, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#18,17,2) AS sun_sales#25, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#19,17,2) AS mon_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#20,17,2) AS tue_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#21,17,2) AS wed_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#22,17,2) AS thu_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#23,17,2) AS fri_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#24,17,2) AS sat_sales#31] -(21) TakeOrderedAndProject -Input [9]: [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] -Arguments: 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#32 ASC NULLS FIRST, mon_sales#33 ASC NULLS FIRST, tue_sales#34 ASC NULLS FIRST, wed_sales#35 ASC NULLS FIRST, thu_sales#36 ASC NULLS FIRST, fri_sales#37 ASC NULLS FIRST, sat_sales#38 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] +(19) TakeOrderedAndProject +Input [9]: [s_store_name#9, s_store_id#8, sun_sales#25, mon_sales#26, tue_sales#27, wed_sales#28, thu_sales#29, fri_sales#30, sat_sales#31] +Arguments: 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#25 ASC NULLS FIRST, mon_sales#26 ASC NULLS FIRST, tue_sales#27 ASC NULLS FIRST, wed_sales#28 ASC NULLS FIRST, thu_sales#29 ASC NULLS FIRST, fri_sales#30 ASC NULLS FIRST, sat_sales#31 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#25, mon_sales#26, tue_sales#27, wed_sales#28, thu_sales#29, fri_sales#30, sat_sales#31] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt index ef20430969..d29a65bd4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt @@ -1,31 +1,23 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [s_store_name,s_store_id] #1 - WholeStageCodegen (3) - HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [d_day_name,ss_sales_price,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [d_day_name,ss_store_sk,ss_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] + CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [d_day_name,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_day_name] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk,s_store_id,s_store_name] - CometFilter [s_gmt_offset,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + CometBroadcastExchange #2 + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [s_store_sk,s_store_id,s_store_name] + CometFilter [s_gmt_offset,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index d0d74569be..f128499e34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -6,28 +6,28 @@ TakeOrderedAndProject (36) +- * Project (32) +- * Filter (31) +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.customer (4) - : : : +- BroadcastExchange (13) - : : : +- * ColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (16) - : +- BroadcastExchange (22) - : +- * ColumnarToRow (21) + :- * ColumnarToRow (24) + : +- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.customer (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.customer_address (8) + : : +- CometBroadcastExchange (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.date_dim (13) + : +- CometBroadcastExchange (21) : +- CometFilter (20) : +- CometScan parquet spark_catalog.default.item (19) +- BroadcastExchange (29) @@ -49,163 +49,166 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [c_customer_sk#7, c_current_addr_sk#8] Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [2]: [c_customer_sk#7, c_current_addr_sk#8] +Arguments: [c_customer_sk#7, c_current_addr_sk#8] -(7) BroadcastExchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] +Right output [2]: [c_customer_sk#7, c_current_addr_sk#8] +Arguments: [ws_bill_customer_sk#3], [c_customer_sk#7], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_bill_customer_sk#3] -Right keys [1]: [c_customer_sk#7] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 6] -Output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] +(7) CometProject Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] +Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] -(10) Scan parquet spark_catalog.default.customer_address +(8) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] Condition : isnotnull(ca_address_sk#9) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] +Arguments: [ca_address_sk#9, ca_city#10, ca_zip#11] -(13) BroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] +Right output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] +Arguments: [c_current_addr_sk#8], [ca_address_sk#9], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_addr_sk#8] -Right keys [1]: [ca_address_sk#9] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 6] -Output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11] +(12) CometProject Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#11] +Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11] -(16) ReusedExchange [Reuses operator id: 41] -Output [1]: [d_date_sk#12] +(13) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#5] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(14) CometFilter +Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] +Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) -(18) Project [codegen id : 6] -Output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11] +(15) CometProject +Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] +Arguments: [d_date_sk#12], [d_date_sk#12] + +(16) CometBroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: [d_date_sk#12] + +(17) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11] +Right output [1]: [d_date_sk#12] +Arguments: [ws_sold_date_sk#5], [d_date_sk#12], Inner, BuildRight + +(18) CometProject Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11, d_date_sk#12] +Arguments: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11], [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11] (19) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#15, i_item_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (20) CometFilter -Input [2]: [i_item_sk#13, i_item_id#14] -Condition : isnotnull(i_item_sk#13) +Input [2]: [i_item_sk#15, i_item_id#16] +Condition : isnotnull(i_item_sk#15) -(21) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] +(21) CometBroadcastExchange +Input [2]: [i_item_sk#15, i_item_id#16] +Arguments: [i_item_sk#15, i_item_id#16] -(22) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(22) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11] +Right output [2]: [i_item_sk#15, i_item_id#16] +Arguments: [ws_item_sk#2], [i_item_sk#15], Inner, BuildRight -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(23) CometProject +Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11, i_item_sk#15, i_item_id#16] +Arguments: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16], [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] -(24) Project [codegen id : 6] -Output [4]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#14] -Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11, i_item_sk#13, i_item_id#14] +(24) ColumnarToRow [codegen id : 2] +Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] (25) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] +Output [2]: [i_item_sk#17, i_item_id#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] ReadSchema: struct (26) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : i_item_sk#15 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#17, i_item_id#18] +Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) (27) CometProject -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: [i_item_id#16], [i_item_id#16] +Input [2]: [i_item_sk#17, i_item_id#18] +Arguments: [i_item_id#18], [i_item_id#18] -(28) ColumnarToRow [codegen id : 5] -Input [1]: [i_item_id#16] +(28) ColumnarToRow [codegen id : 1] +Input [1]: [i_item_id#18] (29) BroadcastExchange -Input [1]: [i_item_id#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] +Input [1]: [i_item_id#18] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#16] +(30) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [i_item_id#16] +Right keys [1]: [i_item_id#18] Join type: ExistenceJoin(exists#1) Join condition: None -(31) Filter [codegen id : 6] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#14, exists#1] +(31) Filter [codegen id : 2] +Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16, exists#1] Condition : (substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#11] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#14, exists#1] +Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16, exists#1] -(33) HashAggregate [codegen id : 6] +(33) HashAggregate [codegen id : 2] Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#11] Keys [2]: [ca_zip#11, ca_city#10] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#17] -Results [3]: [ca_zip#11, ca_city#10, sum#18] +Aggregate Attributes [1]: [sum#19] +Results [3]: [ca_zip#11, ca_city#10, sum#20] (34) Exchange -Input [3]: [ca_zip#11, ca_city#10, sum#18] -Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [3]: [ca_zip#11, ca_city#10, sum#20] +Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(35) HashAggregate [codegen id : 7] -Input [3]: [ca_zip#11, ca_city#10, sum#18] +(35) HashAggregate [codegen id : 3] +Input [3]: [ca_zip#11, ca_city#10, sum#20] Keys [2]: [ca_zip#11, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#19] -Results [3]: [ca_zip#11, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#19,17,2) AS sum(ws_sales_price)#20] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#21] +Results [3]: [ca_zip#11, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#21,17,2) AS sum(ws_sales_price)#22] (36) TakeOrderedAndProject -Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] -Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] +Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#22] +Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#11, ca_city#10, sum(ws_sales_price)#22] ===== Subqueries ===== @@ -218,18 +221,18 @@ BroadcastExchange (41) (37) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#21, d_qoy#22] +Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (38) CometFilter -Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] -Condition : ((((isnotnull(d_qoy#22) AND isnotnull(d_year#21)) AND (d_qoy#22 = 2)) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#12)) +Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] +Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) (39) CometProject -Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] +Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] Arguments: [d_date_sk#12], [d_date_sk#12] (40) ColumnarToRow [codegen id : 1] @@ -237,6 +240,6 @@ Input [1]: [d_date_sk#12] (41) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index 383cbb7e3b..c376c4fcd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -1,23 +1,23 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] InputAdapter Exchange [ca_zip,ca_city] #1 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] BroadcastHashJoin [i_item_id,i_item_id] - Project [ws_sales_price,ca_city,ca_zip,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_sales_price,ca_city,ca_zip] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] + CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] CometFilter [ws_bill_customer_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -28,32 +28,22 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometProject [d_date_sk] CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #3 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #4 + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometProject [i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index e07e2ab242..fb99351a57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -1,43 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) +TakeOrderedAndProject (42) ++- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) :- * Project (35) : +- * BroadcastHashJoin Inner BuildRight (34) : :- * HashAggregate (29) : : +- Exchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) + : : +- * ColumnarToRow (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometProject (20) + : : : +- CometBroadcastHashJoin (19) + : : : :- CometProject (14) + : : : : +- CometBroadcastHashJoin (13) + : : : : :- CometProject (8) + : : : : : +- CometBroadcastHashJoin (7) + : : : : : :- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- CometBroadcastExchange (6) + : : : : : +- CometProject (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : : +- CometBroadcastExchange (12) + : : : : +- CometProject (11) + : : : : +- CometFilter (10) + : : : : +- CometScan parquet spark_catalog.default.store (9) + : : : +- CometBroadcastExchange (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.household_demographics (15) + : : +- CometBroadcastExchange (23) : : +- CometFilter (22) : : +- CometScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (33) : +- * ColumnarToRow (32) : +- CometFilter (31) : +- CometScan parquet spark_catalog.default.customer (30) - +- ReusedExchange (36) + +- BroadcastExchange (39) + +- * ColumnarToRow (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.customer_address (36) (1) Scan parquet spark_catalog.default.store_sales @@ -52,131 +55,132 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#10] +(4) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Arguments: [d_date_sk#10], [d_date_sk#10] -(6) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] + +(7) CometBroadcastHashJoin +Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight + +(8) CometProject Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_city#12] +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#13, s_city#14] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#11, s_city#12] -Condition : (s_city#12 IN (Fairview,Midway) AND isnotnull(s_store_sk#11)) +(10) CometFilter +Input [2]: [s_store_sk#13, s_city#14] +Condition : (s_city#14 IN (Fairview,Midway) AND isnotnull(s_store_sk#13)) -(9) CometProject -Input [2]: [s_store_sk#11, s_city#12] -Arguments: [s_store_sk#11], [s_store_sk#11] +(11) CometProject +Input [2]: [s_store_sk#13, s_city#14] +Arguments: [s_store_sk#13], [s_store_sk#13] -(10) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#11] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#13] +Arguments: [s_store_sk#13] -(11) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(13) CometBroadcastHashJoin +Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +Right output [1]: [s_store_sk#13] +Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11] +(14) CometProject +Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -(14) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] +(15) Scan parquet spark_catalog.default.household_demographics +Output [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : (((hd_dep_count#14 = 4) OR (hd_vehicle_count#15 = 3)) AND isnotnull(hd_demo_sk#13)) - -(16) CometProject -Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#13], [hd_demo_sk#13] +(16) CometFilter +Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] +Condition : (((hd_dep_count#16 = 4) OR (hd_vehicle_count#17 = 3)) AND isnotnull(hd_demo_sk#15)) -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#13] +(17) CometProject +Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] +Arguments: [hd_demo_sk#15], [hd_demo_sk#15] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#15] +Arguments: [hd_demo_sk#15] -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#13] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +Right output [1]: [hd_demo_sk#15] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#15], Inner, BuildRight -(20) Project [codegen id : 5] -Output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#13] +(20) CometProject +Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#15] +Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] (21) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_city#17] +Output [2]: [ca_address_sk#18, ca_city#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] ReadSchema: struct (22) CometFilter -Input [2]: [ca_address_sk#16, ca_city#17] -Condition : (isnotnull(ca_address_sk#16) AND isnotnull(ca_city#17)) +Input [2]: [ca_address_sk#18, ca_city#19] +Condition : (isnotnull(ca_address_sk#18) AND isnotnull(ca_city#19)) -(23) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#16, ca_city#17] - -(24) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_city#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastExchange +Input [2]: [ca_address_sk#18, ca_city#19] +Arguments: [ca_address_sk#18, ca_city#19] -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None +(24) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +Right output [2]: [ca_address_sk#18, ca_city#19] +Arguments: [ss_addr_sk#3], [ca_address_sk#18], Inner, BuildRight -(26) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#16, ca_city#17] +(25) CometProject +Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#18, ca_city#19] +Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] -(27) HashAggregate [codegen id : 5] -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] +(26) CometHashAggregate +Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum#18, sum#19] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] + +(27) ColumnarToRow [codegen id : 1] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] (28) Exchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 8] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] +(29) HashAggregate [codegen id : 4] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#22, sum(UnscaledValue(ss_net_profit#7))#23] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#17 AS bought_city#24, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#22,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#23,17,2) AS profit#26] +Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#19 AS bought_city#24, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#22,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#23,17,2) AS profit#26] (30) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] @@ -189,70 +193,85 @@ ReadSchema: struct + +(37) CometFilter +Input [2]: [ca_address_sk#31, ca_city#32] +Condition : (isnotnull(ca_address_sk#31) AND isnotnull(ca_city#32)) + +(38) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#31, ca_city#32] + +(39) BroadcastExchange +Input [2]: [ca_address_sk#31, ca_city#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 8] +(40) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#28] Right keys [1]: [ca_address_sk#31] Join type: Inner Join condition: NOT (ca_city#32 = bought_city#24) -(38) Project [codegen id : 8] +(41) Project [codegen id : 4] Output [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_city#32] -(39) TakeOrderedAndProject +(42) TakeOrderedAndProject Input [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] Arguments: 100, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#24 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(40) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#33, d_dow#34] +(43) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_dow#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [3]: [d_date_sk#10, d_year#33, d_dow#34] -Condition : ((d_dow#34 IN (6,0) AND d_year#33 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) +(44) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(42) CometProject -Input [3]: [d_date_sk#10, d_year#33, d_dow#34] +(45) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(43) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(44) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt index 04c59a2d35..93e3eb05ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - WholeStageCodegen (8) + WholeStageCodegen (4) Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] @@ -7,18 +7,18 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] InputAdapter Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,37 +29,32 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu CometProject [d_date_sk] CometFilter [d_dow,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_dow,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange #4 + CometProject [s_store_sk] + CometFilter [s_city,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometBroadcastExchange #5 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #6 + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 + BroadcastExchange #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index 6185a64a67..ce5faa952f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -13,23 +13,23 @@ TakeOrderedAndProject (45) : : +- Exchange (22) : : +- * HashAggregate (21) : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) + : : +- * ColumnarToRow (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.store (13) : +- BroadcastExchange (35) @@ -57,10 +57,7 @@ ReadSchema: struct Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] - -(4) Scan parquet spark_catalog.default.store_sales +(3) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -68,39 +65,46 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Condition : (isnotnull(ss_item_sk#4) AND isnotnull(ss_store_sk#5)) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#4] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Right output [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: [i_item_sk#1], [ss_item_sk#4], Inner, BuildRight -(9) Project [codegen id : 4] -Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +(7) CometProject Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7], [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_year#10, d_moy#11] + +(11) CometBroadcastHashJoin +Left output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(12) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] +(12) CometProject Input [8]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] +Arguments: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11], [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] (13) Scan parquet spark_catalog.default.store Output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] @@ -113,145 +117,142 @@ ReadSchema: struct Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] Condition : ((isnotnull(s_store_sk#12) AND isnotnull(s_store_name#13)) AND isnotnull(s_company_name#14)) -(15) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [s_store_sk#12, s_store_name#13, s_company_name#14] -(16) BroadcastExchange -Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] +Right output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [ss_store_sk#5], [s_store_sk#12], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] +(17) CometProject Input [9]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14], [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] -(19) HashAggregate [codegen id : 4] +(18) CometHashAggregate Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [1]: [sum#15] -Results [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] + +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#15] (20) Exchange -Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#15] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] +(21) HashAggregate [codegen id : 2] +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#15] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#17] -Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS _w0#19] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#16] +Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#16,17,2) AS _w0#18] (22) Exchange -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(23) Sort [codegen id : 6] -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +(23) Sort [codegen id : 3] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 (24) Window -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] +(25) Filter [codegen id : 4] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) (26) Window -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] -(27) Filter [codegen id : 22] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) +(27) Filter [codegen id : 13] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(28) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +(28) Project [codegen id : 13] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] (29) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Output [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] -(30) HashAggregate [codegen id : 12] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] -Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] +(30) HashAggregate [codegen id : 6] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] +Keys [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26] +Functions [1]: [sum(UnscaledValue(ss_sales_price#28))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#28))#16] +Results [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(ss_sales_price#28))#16,17,2) AS sum_sales#17] (31) Exchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: hashpartitioning(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 13] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 +(32) Sort [codegen id : 7] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, s_store_name#23 ASC NULLS FIRST, s_company_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST], false, 0 (33) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, s_store_name#23, s_company_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(34) Project [codegen id : 14] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] +(34) Project [codegen id : 8] +Output [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#17 AS sum_sales#30, rn#29] +Input [8]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17, rn#29] (35) BroadcastExchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] +Input [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#30, rn#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] +(36) BroadcastHashJoin [codegen id : 13] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] +Right keys [5]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, (rn#29 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +(37) Project [codegen id : 13] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#30] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#30, rn#29] (38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Output [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] -(39) Sort [codegen id : 20] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 +(39) Sort [codegen id : 11] +Input [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] +Arguments: [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, s_store_name#33 ASC NULLS FIRST, s_company_name#34 ASC NULLS FIRST, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST], false, 0 (40) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +Input [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] +Arguments: [rank(d_year#35, d_moy#36) windowspecdefinition(i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, s_store_name#33, s_company_name#34], [d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] -(41) Project [codegen id : 21] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] +(41) Project [codegen id : 12] +Output [6]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#17 AS sum_sales#38, rn#37] +Input [8]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17, rn#37] (42) BroadcastExchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] +Input [6]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#38, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] +(43) BroadcastHashJoin [codegen id : 13] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] +Right keys [5]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, (rn#37 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +(44) Project [codegen id : 13] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#30 AS psum#39, sum_sales#38 AS nsum#40] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#30, i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#38, rn#37] (45) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#39, nsum#40] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#39, nsum#40] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (49) +- * ColumnarToRow (48) +- CometFilter (47) @@ -274,6 +275,6 @@ Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index 80b8da7b14..d2e615f39b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,74 +8,67 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) + BroadcastExchange #7 + WholeStageCodegen (8) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) + WholeStageCodegen (7) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) + Exchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (6) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] InputAdapter ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #9 + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) + WholeStageCodegen (11) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt index 718f3fb317..6b4d61a5e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt @@ -1,32 +1,32 @@ == Physical Plan == * HashAggregate (28) +- Exchange (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.customer_demographics (10) - : +- BroadcastExchange (20) - : +- * ColumnarToRow (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.customer_address (16) - +- ReusedExchange (23) + +- * ColumnarToRow (26) + +- CometHashAggregate (25) + +- CometProject (24) + +- CometBroadcastHashJoin (23) + :- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.store (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.customer_demographics (8) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.customer_address (13) + +- CometBroadcastExchange (22) + +- CometProject (21) + +- CometFilter (20) + +- CometScan parquet spark_catalog.default.date_dim (19) (1) Scan parquet spark_catalog.default.store_sales @@ -41,122 +41,123 @@ ReadSchema: struct= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) -(3) ColumnarToRow [codegen id : 5] -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(4) Scan parquet spark_catalog.default.store +(3) Scan parquet spark_catalog.default.store Output [1]: [s_store_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [1]: [s_store_sk#9] Condition : isnotnull(s_store_sk#9) -(6) ColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#9] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [1]: [s_store_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [s_store_sk#9] -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Right output [1]: [s_store_sk#9] +Arguments: [ss_store_sk#3], [s_store_sk#9], Inner, BuildRight -(9) Project [codegen id : 5] -Output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +(7) CometProject Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] +Arguments: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -(10) Scan parquet spark_catalog.default.customer_demographics +(8) Scan parquet spark_catalog.default.customer_demographics Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree )),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree ))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College )))] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] Condition : (isnotnull(cd_demo_sk#10) AND ((((cd_marital_status#11 = M) AND (cd_education_status#12 = 4 yr Degree )) OR ((cd_marital_status#11 = D) AND (cd_education_status#12 = 2 yr Degree ))) OR ((cd_marital_status#11 = S) AND (cd_education_status#12 = College )))) -(12) ColumnarToRow [codegen id : 2] -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: ((((((cd_marital_status#11 = M) AND (cd_education_status#12 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#11 = D) AND (cd_education_status#12 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#11 = S) AND (cd_education_status#12 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) +(11) CometBroadcastHashJoin +Left output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Right output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] +Arguments: [ss_cdemo_sk#1], [cd_demo_sk#10], Inner, ((((((cd_marital_status#11 = M) AND (cd_education_status#12 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#11 = D) AND (cd_education_status#12 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#11 = S) AND (cd_education_status#12 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))), BuildRight -(15) Project [codegen id : 5] -Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] +(12) CometProject Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] +Arguments: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7], [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] -(16) Scan parquet spark_catalog.default.customer_address +(13) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] 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, [CO,OH,TX]),In(ca_state, [KY,MN,OR])),In(ca_state, [CA,MS,VA]))] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((ca_state#14 IN (CO,OH,TX) OR ca_state#14 IN (OR,MN,KY)) OR ca_state#14 IN (VA,CA,MS))) -(18) CometProject +(15) CometProject Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] Arguments: [ca_address_sk#13, ca_state#14], [ca_address_sk#13, ca_state#14] -(19) ColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#13, ca_state#14] - -(20) BroadcastExchange +(16) CometBroadcastExchange Input [2]: [ca_address_sk#13, ca_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [ca_address_sk#13, ca_state#14] -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#13] -Join type: Inner -Join condition: ((((ca_state#14 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#14 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#14 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))) +(17) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] +Right output [2]: [ca_address_sk#13, ca_state#14] +Arguments: [ss_addr_sk#2], [ca_address_sk#13], Inner, ((((ca_state#14 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#14 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#14 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))), BuildRight -(22) Project [codegen id : 5] -Output [2]: [ss_quantity#4, ss_sold_date_sk#7] +(18) CometProject Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#13, ca_state#14] +Arguments: [ss_quantity#4, ss_sold_date_sk#7], [ss_quantity#4, ss_sold_date_sk#7] -(23) ReusedExchange [Reuses operator id: 33] -Output [1]: [d_date_sk#16] +(19) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_year#17] +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 + +(20) CometFilter +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) + +(21) CometProject +Input [2]: [d_date_sk#16, d_year#17] +Arguments: [d_date_sk#16], [d_date_sk#16] + +(22) CometBroadcastExchange +Input [1]: [d_date_sk#16] +Arguments: [d_date_sk#16] -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [2]: [ss_quantity#4, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#16] +Arguments: [ss_sold_date_sk#7], [d_date_sk#16], Inner, BuildRight -(25) Project [codegen id : 5] -Output [1]: [ss_quantity#4] +(24) CometProject Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#16] +Arguments: [ss_quantity#4], [ss_quantity#4] -(26) HashAggregate [codegen id : 5] +(25) CometHashAggregate Input [1]: [ss_quantity#4] Keys: [] Functions [1]: [partial_sum(ss_quantity#4)] -Aggregate Attributes [1]: [sum#17] -Results [1]: [sum#18] + +(26) ColumnarToRow [codegen id : 1] +Input [1]: [sum#18] (27) Exchange Input [1]: [sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 2] Input [1]: [sum#18] Keys: [] Functions [1]: [sum(ss_quantity#4)] @@ -174,18 +175,18 @@ BroadcastExchange (33) (29) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_year#21] +Output [2]: [d_date_sk#16, d_year#17] 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 (30) CometFilter -Input [2]: [d_date_sk#16, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) (31) CometProject -Input [2]: [d_date_sk#16, d_year#21] +Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] (32) ColumnarToRow [codegen id : 1] @@ -193,6 +194,6 @@ Input [1]: [d_date_sk#16] (33) BroadcastExchange Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt index 4022da74f1..6ebf6af07a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt @@ -1,19 +1,19 @@ -WholeStageCodegen (6) +WholeStageCodegen (2) HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] InputAdapter Exchange #1 - WholeStageCodegen (5) - HashAggregate [ss_quantity] [sum,sum] - Project [ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_quantity] + CometProject [ss_quantity] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_quantity,ss_sold_date_sk] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + CometProject [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + CometProject [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -24,27 +24,17 @@ WholeStageCodegen (6) CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #3 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #4 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastExchange #5 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index bbb550e057..7df26543a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -1,78 +1,81 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- Union (70) - :- * Project (23) - : +- * Filter (22) - : +- Window (21) - : +- * Sort (20) - : +- Window (19) - : +- * Sort (18) - : +- Exchange (17) - : +- * HashAggregate (16) - : +- Exchange (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.web_returns (5) - : +- ReusedExchange (11) - :- * Project (46) - : +- * Filter (45) - : +- Window (44) - : +- * Sort (43) - : +- Window (42) - : +- * Sort (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometScan parquet spark_catalog.default.catalog_returns (28) - : +- ReusedExchange (34) - +- * Project (69) - +- * Filter (68) - +- Window (67) - +- * Sort (66) - +- Window (65) - +- * Sort (64) - +- Exchange (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * ColumnarToRow (56) - : +- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometBroadcastExchange (50) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan parquet spark_catalog.default.store_sales (47) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.store_returns (51) - +- ReusedExchange (57) +TakeOrderedAndProject (77) ++- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * Sort (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- Exchange (18) + : +- * ColumnarToRow (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * Sort (44) + : +- Exchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * ColumnarToRow (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * Sort (67) + +- Exchange (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * ColumnarToRow (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) Scan parquet spark_catalog.default.web_sales @@ -113,344 +116,351 @@ Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_am (8) CometBroadcastHashJoin Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner +Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft (9) CometProject Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(10) ColumnarToRow [codegen id : 2] -Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +(10) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#13] +(11) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) + +(12) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Arguments: [d_date_sk#13], [d_date_sk#13] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: [d_date_sk#13] -(12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight -(13) Project [codegen id : 2] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] +(15) CometProject Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(14) HashAggregate [codegen id : 2] +(16) CometHashAggregate Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(15) Exchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(17) ColumnarToRow [codegen id : 1] +Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] + +(18) Exchange +Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(16) HashAggregate [codegen id : 3] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(19) HashAggregate [codegen id : 2] +Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] +Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#22, sum(coalesce(ws_quantity#3, 0))#23, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#24, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#25] +Results [3]: [ws_item_sk#1 AS item#26, (cast(sum(coalesce(wr_return_quantity#10, 0))#22 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#23 as decimal(15,4))) AS return_ratio#27, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#24 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#25 as decimal(15,4))) AS currency_ratio#28] -(17) Exchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] +(20) Exchange +Input [3]: [item#26, return_ratio#27, currency_ratio#28] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(18) Sort [codegen id : 4] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 +(21) Sort [codegen id : 3] +Input [3]: [item#26, return_ratio#27, currency_ratio#28] +Arguments: [return_ratio#27 ASC NULLS FIRST], false, 0 -(19) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] +(22) Window +Input [3]: [item#26, return_ratio#27, currency_ratio#28] +Arguments: [rank(return_ratio#27) windowspecdefinition(return_ratio#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#29], [return_ratio#27 ASC NULLS FIRST] -(20) Sort [codegen id : 5] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 4] +Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] +Arguments: [currency_ratio#28 ASC NULLS FIRST], false, 0 -(21) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] +(24) Window +Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] +Arguments: [rank(currency_ratio#28) windowspecdefinition(currency_ratio#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#30], [currency_ratio#28 ASC NULLS FIRST] -(22) Filter [codegen id : 6] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) +(25) Filter [codegen id : 5] +Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] +Condition : ((return_rank#29 <= 10) OR (currency_rank#30 <= 10)) -(23) Project [codegen id : 6] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +(26) Project [codegen id : 5] +Output [5]: [web AS channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] -(24) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +(27) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(25) CometFilter -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) +(28) CometFilter +Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] +Condition : (((((((isnotnull(cs_net_profit#36) AND isnotnull(cs_net_paid#35)) AND isnotnull(cs_quantity#34)) AND (cs_net_profit#36 > 1.00)) AND (cs_net_paid#35 > 0.00)) AND (cs_quantity#34 > 0)) AND isnotnull(cs_order_number#33)) AND isnotnull(cs_item_sk#32)) -(26) CometProject -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(29) CometProject +Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] +Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37], [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] -(27) CometBroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(30) CometBroadcastExchange +Input [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] -(28) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +(31) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(29) CometFilter -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) - -(30) CometProject -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(31) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner - -(32) CometProject -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] - -(33) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] - -(34) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#48] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] - -(37) HashAggregate [codegen id : 8] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] -Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] - -(38) Exchange -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(39) HashAggregate [codegen id : 9] -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] -Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] - -(40) Exchange -Input [3]: [item#65, return_ratio#66, currency_ratio#67] +(32) CometFilter +Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] +Condition : (((isnotnull(cr_return_amount#42) AND (cr_return_amount#42 > 10000.00)) AND isnotnull(cr_order_number#40)) AND isnotnull(cr_item_sk#39)) + +(33) CometProject +Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] +Arguments: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42], [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] + +(34) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +Right output [4]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] +Arguments: [cs_order_number#33, cs_item_sk#32], [cr_order_number#40, cr_item_sk#39], Inner, BuildLeft + +(35) CometProject +Input [9]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] +Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] + +(36) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#44] + +(37) CometBroadcastHashJoin +Left output [6]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] +Right output [1]: [d_date_sk#44] +Arguments: [cs_sold_date_sk#37], [d_date_sk#44], Inner, BuildRight + +(38) CometProject +Input [7]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42, d_date_sk#44] +Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] + +(39) CometHashAggregate +Input [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] +Keys [1]: [cs_item_sk#32] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#41, 0)), partial_sum(coalesce(cs_quantity#34, 0)), partial_sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] + +(40) ColumnarToRow [codegen id : 6] +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] + +(41) Exchange +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] +Arguments: hashpartitioning(cs_item_sk#32, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(42) HashAggregate [codegen id : 7] +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] +Keys [1]: [cs_item_sk#32] +Functions [4]: [sum(coalesce(cr_return_quantity#41, 0)), sum(coalesce(cs_quantity#34, 0)), sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#41, 0))#51, sum(coalesce(cs_quantity#34, 0))#52, sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#53, sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#54] +Results [3]: [cs_item_sk#32 AS item#55, (cast(sum(coalesce(cr_return_quantity#41, 0))#51 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#34, 0))#52 as decimal(15,4))) AS return_ratio#56, (cast(sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#53 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#54 as decimal(15,4))) AS currency_ratio#57] + +(43) Exchange +Input [3]: [item#55, return_ratio#56, currency_ratio#57] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(41) Sort [codegen id : 10] -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 +(44) Sort [codegen id : 8] +Input [3]: [item#55, return_ratio#56, currency_ratio#57] +Arguments: [return_ratio#56 ASC NULLS FIRST], false, 0 -(42) Window -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] +(45) Window +Input [3]: [item#55, return_ratio#56, currency_ratio#57] +Arguments: [rank(return_ratio#56) windowspecdefinition(return_ratio#56 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#58], [return_ratio#56 ASC NULLS FIRST] -(43) Sort [codegen id : 11] -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 9] +Input [4]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58] +Arguments: [currency_ratio#57 ASC NULLS FIRST], false, 0 -(44) Window -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] +(47) Window +Input [4]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58] +Arguments: [rank(currency_ratio#57) windowspecdefinition(currency_ratio#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#59], [currency_ratio#57 ASC NULLS FIRST] -(45) Filter [codegen id : 12] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] -Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) +(48) Filter [codegen id : 10] +Input [5]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58, currency_rank#59] +Condition : ((return_rank#58 <= 10) OR (currency_rank#59 <= 10)) -(46) Project [codegen id : 12] -Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +(49) Project [codegen id : 10] +Output [5]: [catalog AS channel#60, item#55, return_ratio#56, return_rank#58, currency_rank#59] +Input [5]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58, currency_rank#59] -(47) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +(50) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(48) CometFilter -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) +(51) CometFilter +Input [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] +Condition : (((((((isnotnull(ss_net_profit#65) AND isnotnull(ss_net_paid#64)) AND isnotnull(ss_quantity#63)) AND (ss_net_profit#65 > 1.00)) AND (ss_net_paid#64 > 0.00)) AND (ss_quantity#63 > 0)) AND isnotnull(ss_ticket_number#62)) AND isnotnull(ss_item_sk#61)) -(49) CometProject -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(52) CometProject +Input [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] +Arguments: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66], [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] -(50) CometBroadcastExchange -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(53) CometBroadcastExchange +Input [5]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] +Arguments: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] -(51) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +(54) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(52) CometFilter -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) - -(53) CometProject -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(54) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner - -(55) CometProject -Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] - -(56) ColumnarToRow [codegen id : 14] -Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] - -(57) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#83] - -(58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#76] -Right keys [1]: [d_date_sk#83] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 14] -Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] - -(60) HashAggregate [codegen id : 14] -Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Keys [1]: [ss_item_sk#71] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] -Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(61) Exchange -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(62) HashAggregate [codegen id : 15] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Keys [1]: [ss_item_sk#71] -Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] -Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] - -(63) Exchange -Input [3]: [item#100, return_ratio#101, currency_ratio#102] +(55) CometFilter +Input [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] +Condition : (((isnotnull(sr_return_amt#71) AND (sr_return_amt#71 > 10000.00)) AND isnotnull(sr_ticket_number#69)) AND isnotnull(sr_item_sk#68)) + +(56) CometProject +Input [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] +Arguments: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71], [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] + +(57) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] +Right output [4]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] +Arguments: [ss_ticket_number#62, ss_item_sk#61], [sr_ticket_number#69, sr_item_sk#68], Inner, BuildLeft + +(58) CometProject +Input [9]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] +Arguments: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71], [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71] + +(59) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#73] + +(60) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71] +Right output [1]: [d_date_sk#73] +Arguments: [ss_sold_date_sk#66], [d_date_sk#73], Inner, BuildRight + +(61) CometProject +Input [7]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71, d_date_sk#73] +Arguments: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71], [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71] + +(62) CometHashAggregate +Input [5]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71] +Keys [1]: [ss_item_sk#61] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#70, 0)), partial_sum(coalesce(ss_quantity#63, 0)), partial_sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))] + +(63) ColumnarToRow [codegen id : 11] +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] + +(64) Exchange +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] +Arguments: hashpartitioning(ss_item_sk#61, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(65) HashAggregate [codegen id : 12] +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] +Keys [1]: [ss_item_sk#61] +Functions [4]: [sum(coalesce(sr_return_quantity#70, 0)), sum(coalesce(ss_quantity#63, 0)), sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#70, 0))#80, sum(coalesce(ss_quantity#63, 0))#81, sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00))#82, sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))#83] +Results [3]: [ss_item_sk#61 AS item#84, (cast(sum(coalesce(sr_return_quantity#70, 0))#80 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#63, 0))#81 as decimal(15,4))) AS return_ratio#85, (cast(sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00))#82 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))#83 as decimal(15,4))) AS currency_ratio#86] + +(66) Exchange +Input [3]: [item#84, return_ratio#85, currency_ratio#86] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(64) Sort [codegen id : 16] -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 +(67) Sort [codegen id : 13] +Input [3]: [item#84, return_ratio#85, currency_ratio#86] +Arguments: [return_ratio#85 ASC NULLS FIRST], false, 0 -(65) Window -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] +(68) Window +Input [3]: [item#84, return_ratio#85, currency_ratio#86] +Arguments: [rank(return_ratio#85) windowspecdefinition(return_ratio#85 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#87], [return_ratio#85 ASC NULLS FIRST] -(66) Sort [codegen id : 17] -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 +(69) Sort [codegen id : 14] +Input [4]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87] +Arguments: [currency_ratio#86 ASC NULLS FIRST], false, 0 -(67) Window -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] +(70) Window +Input [4]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87] +Arguments: [rank(currency_ratio#86) windowspecdefinition(currency_ratio#86 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#88], [currency_ratio#86 ASC NULLS FIRST] -(68) Filter [codegen id : 18] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) +(71) Filter [codegen id : 15] +Input [5]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87, currency_rank#88] +Condition : ((return_rank#87 <= 10) OR (currency_rank#88 <= 10)) -(69) Project [codegen id : 18] -Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] +(72) Project [codegen id : 15] +Output [5]: [store AS channel#89, item#84, return_ratio#85, return_rank#87, currency_rank#88] +Input [5]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87, currency_rank#88] -(70) Union +(73) Union -(71) HashAggregate [codegen id : 19] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(74) HashAggregate [codegen id : 16] +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(72) Exchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(75) Exchange +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Arguments: hashpartitioning(channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(73) HashAggregate [codegen id : 20] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(76) HashAggregate [codegen id : 17] +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(74) TakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(77) TakeOrderedAndProject +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Arguments: 100, [channel#31 ASC NULLS FIRST, return_rank#29 ASC NULLS FIRST, currency_rank#30 ASC NULLS FIRST], [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan parquet spark_catalog.default.date_dim (75) +BroadcastExchange (82) ++- * ColumnarToRow (81) + +- CometProject (80) + +- CometFilter (79) + +- CometScan parquet spark_catalog.default.date_dim (78) -(75) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#106, d_moy#107] +(78) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] -Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) +(79) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(77) CometProject -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +(80) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(78) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(79) BroadcastExchange +(82) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index 43ebf34cc0..8d7b158d0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -1,35 +1,35 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (20) + WholeStageCodegen (17) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (19) + WholeStageCodegen (16) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (5) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (5) + WholeStageCodegen (4) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (4) + WholeStageCodegen (3) Sort [return_ratio] InputAdapter Exchange #2 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ws_item_sk] #3 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] CometBroadcastExchange #4 @@ -47,34 +47,36 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_return_amt,wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (12) + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (11) + WholeStageCodegen (9) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (10) + WholeStageCodegen (8) Sort [return_ratio] InputAdapter - Exchange #6 - WholeStageCodegen (9) + Exchange #7 + WholeStageCodegen (7) HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [cs_item_sk] #7 - WholeStageCodegen (8) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [cs_item_sk] #8 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #8 + CometBroadcastExchange #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] @@ -82,34 +84,33 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_return_amount,cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (18) + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (15) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (17) + WholeStageCodegen (14) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (16) + WholeStageCodegen (13) Sort [return_ratio] InputAdapter - Exchange #9 - WholeStageCodegen (15) + Exchange #10 + WholeStageCodegen (12) HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [ss_item_sk] #11 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #11 + CometBroadcastExchange #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] @@ -117,5 +118,4 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index e3f7538d1c..07bd5f03a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -5,68 +5,68 @@ TakeOrderedAndProject (70) +- * HashAggregate (67) +- * Expand (66) +- Union (65) - :- * HashAggregate (20) - : +- Exchange (19) - : +- * HashAggregate (18) - : +- * Project (17) - : +- * BroadcastHashJoin Inner BuildRight (16) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (8) - : : : +- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : +- ReusedExchange (9) - : +- BroadcastExchange (15) - : +- * ColumnarToRow (14) - : +- CometFilter (13) - : +- CometScan parquet spark_catalog.default.store (12) - :- * HashAggregate (40) - : +- Exchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- CometUnion (27) - : : : :- CometProject (23) - : : : : +- CometFilter (22) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (24) - : : +- ReusedExchange (29) - : +- BroadcastExchange (35) - : +- * ColumnarToRow (34) - : +- CometFilter (33) - : +- CometScan parquet spark_catalog.default.catalog_page (32) + :- * HashAggregate (22) + : +- Exchange (21) + : +- * ColumnarToRow (20) + : +- CometHashAggregate (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometUnion (7) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : +- CometBroadcastExchange (11) + : : +- CometProject (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.date_dim (8) + : +- CometBroadcastExchange (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.store (14) + :- * HashAggregate (41) + : +- Exchange (40) + : +- * ColumnarToRow (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometUnion (29) + : : : :- CometProject (25) + : : : : +- CometFilter (24) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (23) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) + : : +- ReusedExchange (30) + : +- CometBroadcastExchange (35) + : +- CometFilter (34) + : +- CometScan parquet spark_catalog.default.catalog_page (33) +- * HashAggregate (64) +- Exchange (63) - +- * HashAggregate (62) - +- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * Project (55) - : +- * BroadcastHashJoin Inner BuildRight (54) - : :- * ColumnarToRow (52) - : : +- CometUnion (51) - : : :- CometProject (43) - : : : +- CometFilter (42) - : : : +- CometScan parquet spark_catalog.default.web_sales (41) - : : +- CometProject (50) - : : +- CometBroadcastHashJoin (49) - : : :- CometBroadcastExchange (45) - : : : +- CometScan parquet spark_catalog.default.web_returns (44) - : : +- CometProject (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.web_sales (46) - : +- ReusedExchange (53) - +- BroadcastExchange (59) - +- * ColumnarToRow (58) + +- * ColumnarToRow (62) + +- CometHashAggregate (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometUnion (52) + : : :- CometProject (44) + : : : +- CometFilter (43) + : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometBroadcastExchange (46) + : : : +- CometScan parquet spark_catalog.default.web_returns (45) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan parquet spark_catalog.default.web_sales (47) + : +- ReusedExchange (53) + +- CometBroadcastExchange (58) +- CometFilter (57) +- CometScan parquet spark_catalog.default.web_site (56) @@ -107,310 +107,305 @@ Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, n Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] -(8) ColumnarToRow [codegen id : 3] -Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] +ReadSchema: struct + +(9) CometFilter +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(9) ReusedExchange [Reuses operator id: 75] -Output [1]: [d_date_sk#22] +(10) CometProject +Input [2]: [d_date_sk#22, d_date#23] +Arguments: [d_date_sk#22], [d_date_sk#22] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: [d_date_sk#22] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [1]: [d_date_sk#22] +Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight -(11) Project [codegen id : 3] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +(13) CometProject Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] +Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -(12) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] +(14) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_store_id#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(13) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) +(15) CometFilter +Input [2]: [s_store_sk#24, s_store_id#25] +Condition : isnotnull(s_store_sk#24) -(14) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#23, s_store_id#24] +(16) CometBroadcastExchange +Input [2]: [s_store_sk#24, s_store_id#25] +Arguments: [s_store_sk#24, s_store_id#25] -(15) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(17) CometBroadcastHashJoin +Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [2]: [s_store_sk#24, s_store_id#25] +Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None +(18) CometProject +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] +Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -(17) Project [codegen id : 3] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#24] - -(18) HashAggregate [codegen id : 3] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Keys [1]: [s_store_id#24] +(19) CometHashAggregate +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] -Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -(19) Exchange -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(20) ColumnarToRow [codegen id : 1] +Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] + +(21) Exchange +Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Keys [1]: [s_store_id#24] +(22) HashAggregate [codegen id : 2] +Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] +Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#39, store channel AS channel#40, concat(store, s_store_id#24) AS id#41] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#30, sum(UnscaledValue(return_amt#10))#31, sum(UnscaledValue(profit#9))#32, sum(UnscaledValue(net_loss#11))#33] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#30,17,2) AS sales#34, MakeDecimal(sum(UnscaledValue(return_amt#10))#31,17,2) AS returns#35, (MakeDecimal(sum(UnscaledValue(profit#9))#32,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#33,17,2)) AS profit#36, store channel AS channel#37, concat(store, s_store_id#25) AS id#38] -(21) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(23) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(22) CometFilter -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : isnotnull(cs_catalog_page_sk#42) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +Condition : isnotnull(cs_catalog_page_sk#39) -(23) CometProject -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52], [cs_catalog_page_sk#42 AS page_sk#47, cs_sold_date_sk#45 AS date_sk#48, cs_ext_sales_price#43 AS sales_price#49, cs_net_profit#44 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] +(25) CometProject +Input [4]: [cs_catalog_page_sk#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +Arguments: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49], [cs_catalog_page_sk#39 AS page_sk#44, cs_sold_date_sk#42 AS date_sk#45, cs_ext_sales_price#40 AS sales_price#46, cs_net_profit#41 AS profit#47, 0.00 AS return_amt#48, 0.00 AS net_loss#49] -(24) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +(26) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#50, cr_return_amount#51, cr_net_loss#52, cr_returned_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(cr_returned_date_sk#53), dynamicpruningexpression(cr_returned_date_sk#53 IN dynamicpruning#43)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Condition : isnotnull(cr_catalog_page_sk#53) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#50, cr_return_amount#51, cr_net_loss#52, cr_returned_date_sk#53] +Condition : isnotnull(cr_catalog_page_sk#50) -(26) CometProject -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Arguments: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62], [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] +(28) CometProject +Input [4]: [cr_catalog_page_sk#50, cr_return_amount#51, cr_net_loss#52, cr_returned_date_sk#53] +Arguments: [page_sk#54, date_sk#55, sales_price#56, profit#57, return_amt#58, net_loss#59], [cr_catalog_page_sk#50 AS page_sk#54, cr_returned_date_sk#53 AS date_sk#55, 0.00 AS sales_price#56, 0.00 AS profit#57, cr_return_amount#51 AS return_amt#58, cr_net_loss#52 AS net_loss#59] -(27) CometUnion -Child 0 Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] -Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62] +(29) CometUnion +Child 0 Input [6]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49] +Child 1 Input [6]: [page_sk#54, date_sk#55, sales_price#56, profit#57, return_amt#58, net_loss#59] -(28) ColumnarToRow [codegen id : 7] -Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#60] -(29) ReusedExchange [Reuses operator id: 75] -Output [1]: [d_date_sk#63] +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49] +Right output [1]: [d_date_sk#60] +Arguments: [date_sk#45], [d_date_sk#60], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [date_sk#48] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None +(32) CometProject +Input [7]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49, d_date_sk#60] +Arguments: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49], [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49] -(31) Project [codegen id : 7] -Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] -Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] - -(32) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +(33) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(33) CometFilter -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) - -(34) ColumnarToRow [codegen id : 6] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] - -(35) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 7] -Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] -Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#65] - -(38) HashAggregate [codegen id : 7] -Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] -Keys [1]: [cp_catalog_page_id#65] -Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum#66, sum#67, sum#68, sum#69] -Results [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] - -(39) Exchange -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(40) HashAggregate [codegen id : 8] -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Keys [1]: [cp_catalog_page_id#65] -Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#49))#74,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(return_amt#51))#75,17,2) AS returns#79, (MakeDecimal(sum(UnscaledValue(profit#50))#76,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#77,17,2)) AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#65) AS id#82] - -(41) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +(34) CometFilter +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) + +(35) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [cp_catalog_page_sk#61, cp_catalog_page_id#62] + +(36) CometBroadcastHashJoin +Left output [5]: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49] +Right output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [page_sk#44], [cp_catalog_page_sk#61], Inner, BuildRight + +(37) CometProject +Input [7]: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62], [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62] + +(38) CometHashAggregate +Input [5]: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] +Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(UnscaledValue(return_amt#48)), partial_sum(UnscaledValue(profit#47)), partial_sum(UnscaledValue(net_loss#49))] + +(39) ColumnarToRow [codegen id : 3] +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] + +(40) Exchange +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(41) HashAggregate [codegen id : 4] +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] +Keys [1]: [cp_catalog_page_id#62] +Functions [4]: [sum(UnscaledValue(sales_price#46)), sum(UnscaledValue(return_amt#48)), sum(UnscaledValue(profit#47)), sum(UnscaledValue(net_loss#49))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#46))#67, sum(UnscaledValue(return_amt#48))#68, sum(UnscaledValue(profit#47))#69, sum(UnscaledValue(net_loss#49))#70] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#46))#67,17,2) AS sales#71, MakeDecimal(sum(UnscaledValue(return_amt#48))#68,17,2) AS returns#72, (MakeDecimal(sum(UnscaledValue(profit#47))#69,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#49))#70,17,2)) AS profit#73, catalog channel AS channel#74, concat(catalog_page, cp_catalog_page_id#62) AS id#75] + +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#80)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(42) CometFilter -Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] -Condition : isnotnull(ws_web_site_sk#83) +(43) CometFilter +Input [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_web_site_sk#76) -(43) CometProject -Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] -Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] +(44) CometProject +Input [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] +Arguments: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86], [ws_web_site_sk#76 AS wsr_web_site_sk#81, ws_sold_date_sk#79 AS date_sk#82, ws_ext_sales_price#77 AS sales_price#83, ws_net_profit#78 AS profit#84, 0.00 AS return_amt#85, 0.00 AS net_loss#86] -(44) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +(45) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#80)] ReadSchema: struct -(45) CometBroadcastExchange -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Arguments: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -(46) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +(47) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(47) CometFilter -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) - -(48) CometProject -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +(48) CometFilter +Input [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] +Condition : ((isnotnull(ws_item_sk#92) AND isnotnull(ws_order_number#94)) AND isnotnull(ws_web_site_sk#93)) -(49) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner +(49) CometProject +Input [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] +Arguments: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94], [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] -(50) CometProject -Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Right output [3]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] +Arguments: [wr_item_sk#87, wr_order_number#88], [ws_item_sk#92, ws_order_number#94], Inner, BuildLeft -(51) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] +(51) CometProject +Input [8]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] +Arguments: [wsr_web_site_sk#96, date_sk#97, sales_price#98, profit#99, return_amt#100, net_loss#101], [ws_web_site_sk#93 AS wsr_web_site_sk#96, wr_returned_date_sk#91 AS date_sk#97, 0.00 AS sales_price#98, 0.00 AS profit#99, wr_return_amt#89 AS return_amt#100, wr_net_loss#90 AS net_loss#101] -(52) ColumnarToRow [codegen id : 11] -Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86] +Child 1 Input [6]: [wsr_web_site_sk#96, date_sk#97, sales_price#98, profit#99, return_amt#100, net_loss#101] -(53) ReusedExchange [Reuses operator id: 75] -Output [1]: [d_date_sk#109] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#102] -(54) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#89] -Right keys [1]: [d_date_sk#109] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86] +Right output [1]: [d_date_sk#102] +Arguments: [date_sk#82], [d_date_sk#102], Inner, BuildRight -(55) Project [codegen id : 11] -Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] -Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] +(55) CometProject +Input [7]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86, d_date_sk#102] +Arguments: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86], [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86] (56) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#110, web_site_id#111] +Output [2]: [web_site_sk#103, web_site_id#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (57) CometFilter -Input [2]: [web_site_sk#110, web_site_id#111] -Condition : isnotnull(web_site_sk#110) +Input [2]: [web_site_sk#103, web_site_id#104] +Condition : isnotnull(web_site_sk#103) -(58) ColumnarToRow [codegen id : 10] -Input [2]: [web_site_sk#110, web_site_id#111] +(58) CometBroadcastExchange +Input [2]: [web_site_sk#103, web_site_id#104] +Arguments: [web_site_sk#103, web_site_id#104] -(59) BroadcastExchange -Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(59) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86] +Right output [2]: [web_site_sk#103, web_site_id#104] +Arguments: [wsr_web_site_sk#81], [web_site_sk#103], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [wsr_web_site_sk#88] -Right keys [1]: [web_site_sk#110] -Join type: Inner -Join condition: None +(60) CometProject +Input [7]: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_sk#103, web_site_id#104] +Arguments: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104], [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104] -(61) Project [codegen id : 11] -Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] -Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] +(61) CometHashAggregate +Input [5]: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104] +Keys [1]: [web_site_id#104] +Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(UnscaledValue(return_amt#85)), partial_sum(UnscaledValue(profit#84)), partial_sum(UnscaledValue(net_loss#86))] -(62) HashAggregate [codegen id : 11] -Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] -Keys [1]: [web_site_id#111] -Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] -Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] +(62) ColumnarToRow [codegen id : 5] +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] (63) Exchange -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(64) HashAggregate [codegen id : 12] -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Keys [1]: [web_site_id#111] -Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#126, web channel AS channel#127, concat(web_site, web_site_id#111) AS id#128] +(64) HashAggregate [codegen id : 6] +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Keys [1]: [web_site_id#104] +Functions [4]: [sum(UnscaledValue(sales_price#83)), sum(UnscaledValue(return_amt#85)), sum(UnscaledValue(profit#84)), sum(UnscaledValue(net_loss#86))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#83))#109, sum(UnscaledValue(return_amt#85))#110, sum(UnscaledValue(profit#84))#111, sum(UnscaledValue(net_loss#86))#112] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#83))#109,17,2) AS sales#113, MakeDecimal(sum(UnscaledValue(return_amt#85))#110,17,2) AS returns#114, (MakeDecimal(sum(UnscaledValue(profit#84))#111,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#86))#112,17,2)) AS profit#115, web channel AS channel#116, concat(web_site, web_site_id#104) AS id#117] (65) Union -(66) Expand [codegen id : 13] -Input [5]: [sales#37, returns#38, profit#39, channel#40, id#41] -Arguments: [[sales#37, returns#38, profit#39, channel#40, id#41, 0], [sales#37, returns#38, profit#39, channel#40, null, 1], [sales#37, returns#38, profit#39, null, null, 3]], [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] +(66) Expand [codegen id : 7] +Input [5]: [sales#34, returns#35, profit#36, channel#37, id#38] +Arguments: [[sales#34, returns#35, profit#36, channel#37, id#38, 0], [sales#34, returns#35, profit#36, channel#37, null, 1], [sales#34, returns#35, profit#36, null, null, 3]], [sales#34, returns#35, profit#36, channel#118, id#119, spark_grouping_id#120] -(67) HashAggregate [codegen id : 13] -Input [6]: [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] -Keys [3]: [channel#129, id#130, spark_grouping_id#131] -Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -Results [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +(67) HashAggregate [codegen id : 7] +Input [6]: [sales#34, returns#35, profit#36, channel#118, id#119, spark_grouping_id#120] +Keys [3]: [channel#118, id#119, spark_grouping_id#120] +Functions [3]: [partial_sum(sales#34), partial_sum(returns#35), partial_sum(profit#36)] +Aggregate Attributes [6]: [sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] +Results [9]: [channel#118, id#119, spark_grouping_id#120, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] (68) Exchange -Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#129, id#130, spark_grouping_id#131, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [9]: [channel#118, id#119, spark_grouping_id#120, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Arguments: hashpartitioning(channel#118, id#119, spark_grouping_id#120, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(69) HashAggregate [codegen id : 14] -Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [3]: [channel#129, id#130, spark_grouping_id#131] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#144, sum(returns#38)#145, sum(profit#39)#146] -Results [5]: [channel#129, id#130, sum(sales#37)#144 AS sales#147, sum(returns#38)#145 AS returns#148, sum(profit#39)#146 AS profit#149] +(69) HashAggregate [codegen id : 8] +Input [9]: [channel#118, id#119, spark_grouping_id#120, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Keys [3]: [channel#118, id#119, spark_grouping_id#120] +Functions [3]: [sum(sales#34), sum(returns#35), sum(profit#36)] +Aggregate Attributes [3]: [sum(sales#34)#133, sum(returns#35)#134, sum(profit#36)#135] +Results [5]: [channel#118, id#119, sum(sales#34)#133 AS sales#136, sum(returns#35)#134 AS returns#137, sum(profit#36)#135 AS profit#138] (70) TakeOrderedAndProject -Input [5]: [channel#129, id#130, sales#147, returns#148, profit#149] -Arguments: 100, [channel#129 ASC NULLS FIRST, id#130 ASC NULLS FIRST], [channel#129, id#130, sales#147, returns#148, profit#149] +Input [5]: [channel#118, id#119, sales#136, returns#137, profit#138] +Arguments: 100, [channel#118 ASC NULLS FIRST, id#119 ASC NULLS FIRST], [channel#118, id#119, sales#136, returns#137, profit#138] ===== Subqueries ===== @@ -423,18 +418,18 @@ BroadcastExchange (75) (71) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#150] +Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct (72) CometFilter -Input [2]: [d_date_sk#22, d_date#150] -Condition : (((isnotnull(d_date#150) AND (d_date#150 >= 2000-08-23)) AND (d_date#150 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) (73) CometProject -Input [2]: [d_date_sk#22, d_date#150] +Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] (74) ColumnarToRow [codegen id : 1] @@ -442,16 +437,16 @@ Input [1]: [d_date_sk#22] (75) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#53 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index 3d539d5917..c4ea8fe24b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (14) + WholeStageCodegen (8) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (13) + WholeStageCodegen (7) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter Exchange [s_store_id] #2 - WholeStageCodegen (3) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,s_store_sk] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [ss_store_sk] @@ -36,27 +36,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (8) + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #5 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter - Exchange [cp_catalog_page_id] #5 - WholeStageCodegen (7) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [cp_catalog_page_id] #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [cs_catalog_page_sk] @@ -66,27 +64,22 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [cr_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (12) + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #7 + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] InputAdapter - Exchange [web_site_id] #7 - WholeStageCodegen (11) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [web_site_id] #8 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [ws_web_site_sk] @@ -94,18 +87,13 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #8 + CometBroadcastExchange #9 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #10 + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt index 0182e0ac1e..c1c7074be6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt @@ -2,31 +2,31 @@ TakeOrderedAndProject (28) +- * HashAggregate (27) +- Exchange (26) - +- * HashAggregate (25) - +- * Project (24) - +- * BroadcastHashJoin Inner BuildRight (23) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.date_dim (16) - +- ReusedExchange (22) + +- * ColumnarToRow (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometProject (17) + : +- CometBroadcastHashJoin (16) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.store_returns (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.store (8) + : +- CometBroadcastExchange (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.date_dim (13) + +- CometBroadcastExchange (21) + +- CometProject (20) + +- CometFilter (19) + +- CometScan parquet spark_catalog.default.date_dim (18) (1) Scan parquet spark_catalog.default.store_sales @@ -41,10 +41,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Arguments: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -(7) BroadcastExchange -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(input[2, int, false], input[0, int, false], input[1, int, false]),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Right output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Arguments: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2], [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 5] -Left keys [3]: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 5] -Output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] +(7) CometProject Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Arguments: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9], [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] -(10) Scan parquet spark_catalog.default.store +(8) Scan parquet spark_catalog.default.store Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Condition : isnotnull(s_store_sk#11) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -(13) BroadcastExchange -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] +Right output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +(12) CometProject Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -(16) Scan parquet spark_catalog.default.date_dim +(13) Scan parquet spark_catalog.default.date_dim Output [1]: [d_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [1]: [d_date_sk#22] Condition : isnotnull(d_date_sk#22) -(18) ColumnarToRow [codegen id : 3] -Input [1]: [d_date_sk#22] - -(19) BroadcastExchange +(15) CometBroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [d_date_sk#22] -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Right output [1]: [d_date_sk#22] +Arguments: [ss_sold_date_sk#5], [d_date_sk#22], Inner, BuildRight -(21) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +(17) CometProject Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#22] +Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] + +(18) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#23, d_year#24, d_moy#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] +ReadSchema: struct -(22) ReusedExchange [Reuses operator id: 33] -Output [1]: [d_date_sk#23] +(19) CometFilter +Input [3]: [d_date_sk#23, d_year#24, d_moy#25] +Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2001)) AND (d_moy#25 = 8)) AND isnotnull(d_date_sk#23)) -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_returned_date_sk#9] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None +(20) CometProject +Input [3]: [d_date_sk#23, d_year#24, d_moy#25] +Arguments: [d_date_sk#23], [d_date_sk#23] -(24) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +(21) CometBroadcastExchange +Input [1]: [d_date_sk#23] +Arguments: [d_date_sk#23] + +(22) CometBroadcastHashJoin +Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Right output [1]: [d_date_sk#23] +Arguments: [sr_returned_date_sk#9], [d_date_sk#23], Inner, BuildRight + +(23) CometProject Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#23] +Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -(25) HashAggregate [codegen id : 5] +(24) CometHashAggregate Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum#24, sum#25, sum#26, sum#27, sum#28] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] + +(25) ColumnarToRow [codegen id : 1] +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#26, sum#27, sum#28, sum#29, sum#30] (26) Exchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#26, sum#27, sum#28, sum#29, sum#30] +Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(27) HashAggregate [codegen id : 6] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] +(27) HashAggregate [codegen id : 2] +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#26, sum#27, sum#28, sum#29, sum#30] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34 AS 30 days #39, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35 AS 31 - 60 days #40, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36 AS 61 - 90 days #41, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37 AS 91 - 120 days #42, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38 AS >120 days #43] +Aggregate Attributes [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#31, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#32, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#33, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#34, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#35] +Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#31 AS 30 days #36, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#32 AS 31 - 60 days #37, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#33 AS 61 - 90 days #38, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#34 AS 91 - 120 days #39, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#35 AS >120 days #40] (28) TakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] -Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] +Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 +Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (33) +- * ColumnarToRow (32) +- CometProject (31) @@ -175,18 +176,18 @@ BroadcastExchange (33) (29) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#23, d_year#44, d_moy#45] +Output [3]: [d_date_sk#23, d_year#24, d_moy#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] ReadSchema: struct (30) CometFilter -Input [3]: [d_date_sk#23, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 8)) AND isnotnull(d_date_sk#23)) +Input [3]: [d_date_sk#23, d_year#24, d_moy#25] +Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2001)) AND (d_moy#25 = 8)) AND isnotnull(d_date_sk#23)) (31) CometProject -Input [3]: [d_date_sk#23, d_year#44, d_moy#45] +Input [3]: [d_date_sk#23, d_year#24, d_moy#25] Arguments: [d_date_sk#23], [d_date_sk#23] (32) ColumnarToRow [codegen id : 1] @@ -194,6 +195,6 @@ Input [1]: [d_date_sk#23] (33) BroadcastExchange Input [1]: [d_date_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt index dfdcaf4975..473b9cdd43 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt @@ -1,50 +1,40 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter Exchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 - WholeStageCodegen (5) - HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] + CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] - CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #2 + CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #4 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometBroadcastExchange #5 + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt index 2613551f0c..bc106394ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt @@ -1,41 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (37) -+- * Filter (36) - +- Window (35) - +- * Sort (34) - +- Exchange (33) - +- * Project (32) - +- * SortMergeJoin FullOuter (31) - :- * Sort (15) - : +- Exchange (14) - : +- * Project (13) - : +- Window (12) - : +- * Sort (11) - : +- Exchange (10) - : +- * HashAggregate (9) - : +- Exchange (8) - : +- * HashAggregate (7) - : +- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- * Sort (30) - +- Exchange (29) - +- * Project (28) - +- Window (27) - +- * Sort (26) - +- Exchange (25) - +- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.store_sales (16) - +- ReusedExchange (19) +TakeOrderedAndProject (40) ++- * Filter (39) + +- Window (38) + +- * Sort (37) + +- Exchange (36) + +- * Project (35) + +- * SortMergeJoin FullOuter (34) + :- * Sort (18) + : +- Exchange (17) + : +- * Project (16) + : +- Window (15) + : +- * Sort (14) + : +- Exchange (13) + : +- * HashAggregate (12) + : +- Exchange (11) + : +- * ColumnarToRow (10) + : +- CometHashAggregate (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.date_dim (3) + +- * Sort (33) + +- Exchange (32) + +- * Project (31) + +- Window (30) + +- * Sort (29) + +- Exchange (28) + +- * HashAggregate (27) + +- Exchange (26) + +- * ColumnarToRow (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometFilter (20) + : +- CometScan parquet spark_catalog.default.store_sales (19) + +- ReusedExchange (21) (1) Scan parquet spark_catalog.default.web_sales @@ -50,65 +53,78 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 42] -Output [2]: [d_date_sk#5, d_date#6] +(4) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] + +(6) CometBroadcastExchange +Input [2]: [d_date_sk#5, d_date#6] +Arguments: [d_date_sk#5, d_date#6] + +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +Right output [2]: [d_date_sk#5, d_date#6] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] +(8) CometProject Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] +Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] -(7) HashAggregate [codegen id : 2] +(9) CometHashAggregate Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] -(8) Exchange +(10) ColumnarToRow [codegen id : 1] +Input [3]: [ws_item_sk#1, d_date#6, sum#8] + +(11) Exchange Input [3]: [ws_item_sk#1, d_date#6, sum#8] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) HashAggregate [codegen id : 3] +(12) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS _w0#11, ws_item_sk#1] -(10) Exchange +(13) Exchange Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) Sort [codegen id : 4] +(14) Sort [codegen id : 3] Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(12) Window +(15) Window Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(13) Project [codegen id : 5] +(16) Project [codegen id : 4] Output [3]: [item_sk#10, d_date#6, cume_sales#12] Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] -(14) Exchange +(17) Exchange Input [3]: [item_sk#10, d_date#6, cume_sales#12] Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(15) Sort [codegen id : 6] +(18) Sort [codegen id : 5] Input [3]: [item_sk#10, d_date#6, cume_sales#12] Arguments: [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(16) Scan parquet spark_catalog.default.store_sales +(19) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] @@ -116,130 +132,127 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(17) CometFilter +(20) CometFilter Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Condition : isnotnull(ss_item_sk#13) -(18) ColumnarToRow [codegen id : 8] -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] - -(19) ReusedExchange [Reuses operator id: 42] +(21) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#17, d_date#18] -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +Right output [2]: [d_date_sk#17, d_date#18] +Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight -(21) Project [codegen id : 8] -Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] +(23) CometProject Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] +Arguments: [ss_item_sk#13, ss_sales_price#14, d_date#18], [ss_item_sk#13, ss_sales_price#14, d_date#18] -(22) HashAggregate [codegen id : 8] +(24) CometHashAggregate Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [ss_item_sk#13, d_date#18, sum#20] -(23) Exchange -Input [3]: [ss_item_sk#13, d_date#18, sum#20] +(25) ColumnarToRow [codegen id : 6] +Input [3]: [ss_item_sk#13, d_date#18, sum#19] + +(26) Exchange +Input [3]: [ss_item_sk#13, d_date#18, sum#19] Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(24) HashAggregate [codegen id : 9] -Input [3]: [ss_item_sk#13, d_date#18, sum#20] +(27) HashAggregate [codegen id : 7] +Input [3]: [ss_item_sk#13, d_date#18, sum#19] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#21] -Results [4]: [ss_item_sk#13 AS item_sk#22, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#21,17,2) AS _w0#23, ss_item_sk#13] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] +Results [4]: [ss_item_sk#13 AS item_sk#21, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS _w0#22, ss_item_sk#13] -(25) Exchange -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] +(28) Exchange +Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(26) Sort [codegen id : 10] -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] +(29) Sort [codegen id : 8] +Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 -(27) Window -Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] -Arguments: [sum(_w0#23) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#24], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] +(30) Window +Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] +Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] -(28) Project [codegen id : 11] -Output [3]: [item_sk#22, d_date#18, cume_sales#24] -Input [5]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13, cume_sales#24] +(31) Project [codegen id : 9] +Output [3]: [item_sk#21, d_date#18, cume_sales#23] +Input [5]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23] -(29) Exchange -Input [3]: [item_sk#22, d_date#18, cume_sales#24] -Arguments: hashpartitioning(item_sk#22, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(32) Exchange +Input [3]: [item_sk#21, d_date#18, cume_sales#23] +Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(30) Sort [codegen id : 12] -Input [3]: [item_sk#22, d_date#18, cume_sales#24] -Arguments: [item_sk#22 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 +(33) Sort [codegen id : 10] +Input [3]: [item_sk#21, d_date#18, cume_sales#23] +Arguments: [item_sk#21 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 13] +(34) SortMergeJoin [codegen id : 11] Left keys [2]: [item_sk#10, d_date#6] -Right keys [2]: [item_sk#22, d_date#18] +Right keys [2]: [item_sk#21, d_date#18] Join type: FullOuter Join condition: None -(32) Project [codegen id : 13] -Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#22 END AS item_sk#25, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#26, cume_sales#12 AS web_sales#27, cume_sales#24 AS store_sales#28] -Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#22, d_date#18, cume_sales#24] +(35) Project [codegen id : 11] +Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#21 END AS item_sk#24, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#25, cume_sales#12 AS web_sales#26, cume_sales#23 AS store_sales#27] +Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#21, d_date#18, cume_sales#23] -(33) Exchange -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: hashpartitioning(item_sk#25, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(36) Exchange +Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(34) Sort [codegen id : 14] -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 12] +Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +Arguments: [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], false, 0 -(35) Window -Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] -Arguments: [max(web_sales#27) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#29, max(store_sales#28) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#30], [item_sk#25], [d_date#26 ASC NULLS FIRST] +(38) Window +Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] -(36) Filter [codegen id : 15] -Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] -Condition : ((isnotnull(web_cumulative#29) AND isnotnull(store_cumulative#30)) AND (web_cumulative#29 > store_cumulative#30)) +(39) Filter [codegen id : 13] +Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) -(37) TakeOrderedAndProject -Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] -Arguments: 100, [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] +(40) TakeOrderedAndProject +Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (42) -+- * ColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan parquet spark_catalog.default.date_dim (38) +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) -(38) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#31] +(41) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(39) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] -Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1200)) AND (d_month_seq#31 <= 1211)) AND isnotnull(d_date_sk#5)) +(42) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(40) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] +(43) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(41) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(42) BroadcastExchange +(45) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index 181cd1b98c..ce6005da6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -1,38 +1,38 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (15) + WholeStageCodegen (13) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (14) + WholeStageCodegen (12) Sort [item_sk,d_date] InputAdapter Exchange [item_sk] #1 - WholeStageCodegen (13) + WholeStageCodegen (11) Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] SortMergeJoin [item_sk,d_date,item_sk,d_date] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (5) Sort [item_sk,d_date] InputAdapter Exchange [item_sk,d_date] #2 - WholeStageCodegen (5) + WholeStageCodegen (4) Project [item_sk,d_date,cume_sales] InputAdapter Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (4) + WholeStageCodegen (3) Sort [ws_item_sk,d_date] InputAdapter Exchange [ws_item_sk] #3 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] InputAdapter Exchange [ws_item_sk,d_date] #4 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -43,33 +43,34 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometProject [d_date_sk,d_date] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + CometBroadcastExchange #6 + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (10) Sort [item_sk,d_date] InputAdapter - Exchange [item_sk,d_date] #6 - WholeStageCodegen (11) + Exchange [item_sk,d_date] #7 + WholeStageCodegen (9) Project [item_sk,d_date,cume_sales] InputAdapter Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (10) + WholeStageCodegen (8) Sort [ss_item_sk,d_date] InputAdapter - Exchange [ss_item_sk] #7 - WholeStageCodegen (9) + Exchange [ss_item_sk] #8 + WholeStageCodegen (7) HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] InputAdapter - Exchange [ss_item_sk,d_date] #8 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [ss_item_sk,d_date] #9 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt index 3d5317eb0c..65c7771256 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt @@ -1,25 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (19) ++- * HashAggregate (18) + +- Exchange (17) + +- * ColumnarToRow (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- CometBroadcastExchange (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.store_sales (4) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.item (9) (1) Scan parquet spark_catalog.default.date_dim @@ -37,10 +35,7 @@ Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) Input [3]: [d_date_sk#1, d_year#2, d_moy#3] Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] -(4) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#1, d_year#2] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -48,78 +43,71 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Condition : isnotnull(ss_item_sk#4) -(7) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [d_date_sk#1, d_year#2] +Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight -(10) Project [codegen id : 3] -Output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +(8) CometProject Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -(11) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] 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 -(12) CometFilter +(10) CometFilter Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) -(13) CometProject +(11) CometProject Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9], [i_item_sk#7, i_brand_id#8, i_brand#9] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9] -(15) BroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#4] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight -(17) Project [codegen id : 3] -Output [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +(14) CometProject Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(18) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#11] -Results [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -(19) Exchange -Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(16) ColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#11] + +(17) Exchange +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#11] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] -Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] +(18) HashAggregate [codegen id : 2] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#11] Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#12] +Results [4]: [d_year#2, i_brand_id#8 AS brand_id#13, i_brand#9 AS brand#14, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#12,17,2) AS ext_price#15] -(21) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#14, brand#15, ext_price#16] -Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, ext_price#16] +(19) TakeOrderedAndProject +Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] +Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt index 91fdc2f17d..557dd3b4ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt @@ -1,31 +1,23 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [d_year,i_brand,i_brand_id] #1 - 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_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] + CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [d_year,ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk,d_year] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 335dc7fa2b..76d80c0438 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -1,32 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * Project (27) - +- * Filter (26) - +- Window (25) - +- * Sort (24) - +- Exchange (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.store (14) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * Sort (25) + +- Exchange (24) + +- * HashAggregate (23) + +- Exchange (22) + +- * ColumnarToRow (21) + +- CometHashAggregate (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- CometBroadcastExchange (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.store_sales (4) + : +- CometBroadcastExchange (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.date_dim (9) + +- CometBroadcastExchange (17) + +- CometFilter (16) + +- CometScan parquet spark_catalog.default.store (15) (1) Scan parquet spark_catalog.default.item @@ -44,10 +45,7 @@ Condition : ((((i_category#4 IN (Books Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] -(4) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#1, i_manufact_id#5] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] @@ -55,140 +53,148 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) -(7) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [i_item_sk#1, i_manufact_id#5] +Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight -(10) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +(8) CometProject Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] + +(9) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(10) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] +Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(11) ReusedExchange [Reuses operator id: 33] -Output [2]: [d_date_sk#15, d_qoy#16] +(11) CometProject +Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] +Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(12) CometBroadcastExchange +Input [2]: [d_date_sk#15, d_qoy#17] +Arguments: [d_date_sk#15, d_qoy#17] -(13) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16] -Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#16] +(13) CometBroadcastHashJoin +Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Right output [2]: [d_date_sk#15, d_qoy#17] +Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(14) Scan parquet spark_catalog.default.store -Output [1]: [s_store_sk#17] +(14) CometProject +Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#17] +Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] + +(15) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#18] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [1]: [s_store_sk#17] -Condition : isnotnull(s_store_sk#17) - -(16) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#17] +(16) CometFilter +Input [1]: [s_store_sk#18] +Condition : isnotnull(s_store_sk#18) -(17) BroadcastExchange -Input [1]: [s_store_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [1]: [s_store_sk#18] +Arguments: [s_store_sk#18] -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#17] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] +Right output [1]: [s_store_sk#18] +Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight -(19) Project [codegen id : 4] -Output [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] -Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16, s_store_sk#17] +(19) CometProject +Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17, s_store_sk#18] +Arguments: [i_manufact_id#5, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_sales_price#12, d_qoy#17] -(20) HashAggregate [codegen id : 4] -Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] -Keys [2]: [i_manufact_id#5, d_qoy#16] +(20) CometHashAggregate +Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#17] +Keys [2]: [i_manufact_id#5, d_qoy#17] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [i_manufact_id#5, d_qoy#16, sum#19] -(21) Exchange -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) ColumnarToRow [codegen id : 1] +Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] + +(22) Exchange +Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] +Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Keys [2]: [i_manufact_id#5, d_qoy#16] +(23) HashAggregate [codegen id : 2] +Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] +Keys [2]: [i_manufact_id#5, d_qoy#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] -(23) Exchange +(24) Exchange Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(24) Sort [codegen id : 6] +(25) Sort [codegen id : 3] Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [i_manufact_id#5 ASC NULLS FIRST], false, 0 -(25) Window +(26) Window Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] -(26) Filter [codegen id : 7] +(27) Filter [codegen id : 4] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END -(27) Project [codegen id : 7] +(28) Project [codegen id : 4] Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -(28) TakeOrderedAndProject +(29) TakeOrderedAndProject Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) -(29) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] +(30) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(30) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) +(31) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] +Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(31) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] +(32) CometProject +Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] +Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(32) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_qoy#16] +(33) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#15, d_qoy#17] -(33) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(34) BroadcastExchange +Input [2]: [d_date_sk#15, d_qoy#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index adda5c34f7..83e53bb665 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -1,51 +1,45 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_manufact_id] InputAdapter Exchange [i_manufact_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_manufact_id,d_qoy] #2 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - Project [i_manufact_id,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_manufact_id,d_qoy,ss_sales_price] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometProject [i_item_sk,i_manufact_id] CometFilter [i_category,i_class,i_brand,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_qoy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 5b291d2402..9eca5ceea9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -1,59 +1,60 @@ == Physical Plan == -TakeOrderedAndProject (55) -+- * HashAggregate (54) - +- Exchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Project (33) - : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : :- * HashAggregate (27) - : : : : +- Exchange (26) - : : : : +- * HashAggregate (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * ColumnarToRow (8) - : : : : : : : +- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (4) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- CometProject (11) - : : : : : : +- CometFilter (10) - : : : : : : +- CometScan parquet spark_catalog.default.item (9) - : : : : : +- ReusedExchange (16) - : : : : +- BroadcastExchange (22) - : : : : +- * ColumnarToRow (21) - : : : : +- CometFilter (20) - : : : : +- CometScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (31) - : : : +- * ColumnarToRow (30) - : : : +- CometFilter (29) - : : : +- CometScan parquet spark_catalog.default.store_sales (28) - : : +- BroadcastExchange (37) - : : +- * ColumnarToRow (36) - : : +- CometFilter (35) - : : +- CometScan parquet spark_catalog.default.customer_address (34) - : +- BroadcastExchange (43) - : +- * ColumnarToRow (42) - : +- CometFilter (41) - : +- CometScan parquet spark_catalog.default.store (40) - +- ReusedExchange (46) +TakeOrderedAndProject (56) ++- * HashAggregate (55) + +- Exchange (54) + +- * HashAggregate (53) + +- * HashAggregate (52) + +- Exchange (51) + +- * HashAggregate (50) + +- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (40) + : : +- * BroadcastHashJoin Inner BuildRight (39) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * HashAggregate (28) + : : : : +- Exchange (27) + : : : : +- * ColumnarToRow (26) + : : : : +- CometHashAggregate (25) + : : : : +- CometProject (24) + : : : : +- CometBroadcastHashJoin (23) + : : : : :- CometProject (19) + : : : : : +- CometBroadcastHashJoin (18) + : : : : : :- CometProject (13) + : : : : : : +- CometBroadcastHashJoin (12) + : : : : : : :- CometUnion (7) + : : : : : : : :- CometProject (3) + : : : : : : : : +- CometFilter (2) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : +- CometProject (6) + : : : : : : : +- CometFilter (5) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (4) + : : : : : : +- CometBroadcastExchange (11) + : : : : : : +- CometProject (10) + : : : : : : +- CometFilter (9) + : : : : : : +- CometScan parquet spark_catalog.default.item (8) + : : : : : +- CometBroadcastExchange (17) + : : : : : +- CometProject (16) + : : : : : +- CometFilter (15) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (14) + : : : : +- CometBroadcastExchange (22) + : : : : +- CometFilter (21) + : : : : +- CometScan parquet spark_catalog.default.customer (20) + : : : +- BroadcastExchange (32) + : : : +- * ColumnarToRow (31) + : : : +- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.store_sales (29) + : : +- BroadcastExchange (38) + : : +- * ColumnarToRow (37) + : : +- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.customer_address (35) + : +- BroadcastExchange (44) + : +- * ColumnarToRow (43) + : +- CometFilter (42) + : +- CometScan parquet spark_catalog.default.store (41) + +- ReusedExchange (47) (1) Scan parquet spark_catalog.default.catalog_sales @@ -92,384 +93,389 @@ Arguments: [sold_date_sk#11, customer_sk#12, item_sk#13], [ws_sold_date_sk#10 AS Child 0 Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] Child 1 Input [3]: [sold_date_sk#11, customer_sk#12, item_sk#13] -(8) ColumnarToRow [codegen id : 4] -Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] - -(9) Scan parquet spark_catalog.default.item +(8) Scan parquet spark_catalog.default.item Output [3]: [i_item_sk#14, i_class#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] ReadSchema: struct -(10) CometFilter +(9) CometFilter Input [3]: [i_item_sk#14, i_class#15, i_category#16] Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women )) AND (i_class#15 = maternity )) AND isnotnull(i_item_sk#14)) -(11) CometProject +(10) CometProject Input [3]: [i_item_sk#14, i_class#15, i_category#16] Arguments: [i_item_sk#14], [i_item_sk#14] -(12) ColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#14] - -(13) BroadcastExchange +(11) CometBroadcastExchange Input [1]: [i_item_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#14] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] +Right output [1]: [i_item_sk#14] +Arguments: [item_sk#7], [i_item_sk#14], Inner, BuildRight -(15) Project [codegen id : 4] -Output [2]: [sold_date_sk#5, customer_sk#6] +(13) CometProject Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] +Arguments: [sold_date_sk#5, customer_sk#6], [sold_date_sk#5, customer_sk#6] -(16) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#17] +(14) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#17, d_year#18, d_moy#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sold_date_sk#5] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(15) CometFilter +Input [3]: [d_date_sk#17, d_year#18, d_moy#19] +Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) -(18) Project [codegen id : 4] -Output [1]: [customer_sk#6] +(16) CometProject +Input [3]: [d_date_sk#17, d_year#18, d_moy#19] +Arguments: [d_date_sk#17], [d_date_sk#17] + +(17) CometBroadcastExchange +Input [1]: [d_date_sk#17] +Arguments: [d_date_sk#17] + +(18) CometBroadcastHashJoin +Left output [2]: [sold_date_sk#5, customer_sk#6] +Right output [1]: [d_date_sk#17] +Arguments: [sold_date_sk#5], [d_date_sk#17], Inner, BuildRight + +(19) CometProject Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] +Arguments: [customer_sk#6], [customer_sk#6] -(19) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] +(20) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#20, c_current_addr_sk#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(20) CometFilter -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) +(21) CometFilter +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] +Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_current_addr_sk#21)) -(21) ColumnarToRow [codegen id : 3] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +(22) CometBroadcastExchange +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] +Arguments: [c_customer_sk#20, c_current_addr_sk#21] -(22) BroadcastExchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(23) CometBroadcastHashJoin +Left output [1]: [customer_sk#6] +Right output [2]: [c_customer_sk#20, c_current_addr_sk#21] +Arguments: [customer_sk#6], [c_customer_sk#20], Inner, BuildRight -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_sk#6] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 4] -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] -Input [3]: [customer_sk#6, c_customer_sk#18, c_current_addr_sk#19] +(24) CometProject +Input [3]: [customer_sk#6, c_customer_sk#20, c_current_addr_sk#21] +Arguments: [c_customer_sk#20, c_current_addr_sk#21], [c_customer_sk#20, c_current_addr_sk#21] -(25) HashAggregate [codegen id : 4] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] +(25) CometHashAggregate +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] +Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] Functions: [] -Aggregate Attributes: [] -Results [2]: [c_customer_sk#18, c_current_addr_sk#19] -(26) Exchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: hashpartitioning(c_customer_sk#18, c_current_addr_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(26) ColumnarToRow [codegen id : 1] +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -(27) HashAggregate [codegen id : 9] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] +(27) Exchange +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] +Arguments: hashpartitioning(c_customer_sk#20, c_current_addr_sk#21, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(28) HashAggregate [codegen id : 6] +Input [2]: [c_customer_sk#20, c_current_addr_sk#21] +Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] Functions: [] Aggregate Attributes: [] -Results [2]: [c_customer_sk#18, c_current_addr_sk#19] +Results [2]: [c_customer_sk#20, c_current_addr_sk#21] -(28) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +(29) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#25)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(29) CometFilter -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#20) +(30) CometFilter +Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] +Condition : isnotnull(ss_customer_sk#22) -(30) ColumnarToRow [codegen id : 5] -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +(31) ColumnarToRow [codegen id : 2] +Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -(31) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(32) BroadcastExchange +Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#18] -Right keys [1]: [ss_customer_sk#20] +(33) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#20] +Right keys [1]: [ss_customer_sk#22] Join type: Inner Join condition: None -(33) Project [codegen id : 9] -Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +(34) Project [codegen id : 6] +Output [4]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] +Input [5]: [c_customer_sk#20, c_current_addr_sk#21, ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -(34) Scan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#24, ca_county#25, ca_state#26] +(35) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#26, ca_county#27, ca_state#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county), IsNotNull(ca_state)] ReadSchema: struct -(35) CometFilter -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnotnull(ca_state#26)) +(36) CometFilter +Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] +Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_county#27)) AND isnotnull(ca_state#28)) -(36) ColumnarToRow [codegen id : 6] -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] +(37) ColumnarToRow [codegen id : 3] +Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -(37) BroadcastExchange -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(38) BroadcastExchange +Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#19] -Right keys [1]: [ca_address_sk#24] +(39) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_current_addr_sk#21] +Right keys [1]: [ca_address_sk#26] Join type: Inner Join condition: None -(39) Project [codegen id : 9] -Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26] -Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#26] +(40) Project [codegen id : 6] +Output [5]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28] +Input [7]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_address_sk#26, ca_county#27, ca_state#28] -(40) Scan parquet spark_catalog.default.store -Output [2]: [s_county#27, s_state#28] +(41) Scan parquet spark_catalog.default.store +Output [2]: [s_county#29, s_state#30] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] ReadSchema: struct -(41) CometFilter -Input [2]: [s_county#27, s_state#28] -Condition : (isnotnull(s_county#27) AND isnotnull(s_state#28)) +(42) CometFilter +Input [2]: [s_county#29, s_state#30] +Condition : (isnotnull(s_county#29) AND isnotnull(s_state#30)) -(42) ColumnarToRow [codegen id : 7] -Input [2]: [s_county#27, s_state#28] +(43) ColumnarToRow [codegen id : 4] +Input [2]: [s_county#29, s_state#30] -(43) BroadcastExchange -Input [2]: [s_county#27, s_state#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [plan_id=6] +(44) BroadcastExchange +Input [2]: [s_county#29, s_state#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [plan_id=4] -(44) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [ca_county#25, ca_state#26] -Right keys [2]: [s_county#27, s_state#28] +(45) BroadcastHashJoin [codegen id : 6] +Left keys [2]: [ca_county#27, ca_state#28] +Right keys [2]: [s_county#29, s_state#30] Join type: Inner Join condition: None -(45) Project [codegen id : 9] -Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26, s_county#27, s_state#28] +(46) Project [codegen id : 6] +Output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] +Input [7]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#28, s_county#29, s_state#30] -(46) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#29] +(47) ReusedExchange [Reuses operator id: 66] +Output [1]: [d_date_sk#31] -(47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#29] +(48) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#24] +Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(48) Project [codegen id : 9] -Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#29] - -(49) HashAggregate [codegen id : 9] -Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Keys [1]: [c_customer_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum#30] -Results [2]: [c_customer_sk#18, sum#31] - -(50) Exchange -Input [2]: [c_customer_sk#18, sum#31] -Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(51) HashAggregate [codegen id : 10] -Input [2]: [c_customer_sk#18, sum#31] -Keys [1]: [c_customer_sk#18] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#32] -Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#32,17,2) / 50) as int) AS segment#33] - -(52) HashAggregate [codegen id : 10] -Input [1]: [segment#33] -Keys [1]: [segment#33] +(49) Project [codegen id : 6] +Output [2]: [c_customer_sk#20, ss_ext_sales_price#23] +Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#31] + +(50) HashAggregate [codegen id : 6] +Input [2]: [c_customer_sk#20, ss_ext_sales_price#23] +Keys [1]: [c_customer_sk#20] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#23))] +Aggregate Attributes [1]: [sum#32] +Results [2]: [c_customer_sk#20, sum#33] + +(51) Exchange +Input [2]: [c_customer_sk#20, sum#33] +Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(52) HashAggregate [codegen id : 7] +Input [2]: [c_customer_sk#20, sum#33] +Keys [1]: [c_customer_sk#20] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#23))#34] +Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#23))#34,17,2) / 50) as int) AS segment#35] + +(53) HashAggregate [codegen id : 7] +Input [1]: [segment#35] +Keys [1]: [segment#35] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [2]: [segment#33, count#35] +Aggregate Attributes [1]: [count#36] +Results [2]: [segment#35, count#37] -(53) Exchange -Input [2]: [segment#33, count#35] -Arguments: hashpartitioning(segment#33, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(54) Exchange +Input [2]: [segment#35, count#37] +Arguments: hashpartitioning(segment#35, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(54) HashAggregate [codegen id : 11] -Input [2]: [segment#33, count#35] -Keys [1]: [segment#33] +(55) HashAggregate [codegen id : 8] +Input [2]: [segment#35, count#37] +Keys [1]: [segment#35] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [3]: [segment#33, count(1)#36 AS num_customers#37, (segment#33 * 50) AS segment_base#38] +Aggregate Attributes [1]: [count(1)#38] +Results [3]: [segment#35, count(1)#38 AS num_customers#39, (segment#35 * 50) AS segment_base#40] -(55) TakeOrderedAndProject -Input [3]: [segment#33, num_customers#37, segment_base#38] -Arguments: 100, [segment#33 ASC NULLS FIRST, num_customers#37 ASC NULLS FIRST], [segment#33, num_customers#37, segment_base#38] +(56) TakeOrderedAndProject +Input [3]: [segment#35, num_customers#39, segment_base#40] +Arguments: 100, [segment#35 ASC NULLS FIRST, num_customers#39 ASC NULLS FIRST], [segment#35, num_customers#39, segment_base#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (60) -+- * ColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.date_dim (56) +BroadcastExchange (61) ++- * ColumnarToRow (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.date_dim (57) -(56) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#39, d_moy#40] +(57) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#17, d_year#18, d_moy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter -Input [3]: [d_date_sk#17, d_year#39, d_moy#40] -Condition : ((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 = 12)) AND (d_year#39 = 1998)) AND isnotnull(d_date_sk#17)) +(58) CometFilter +Input [3]: [d_date_sk#17, d_year#18, d_moy#19] +Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) -(58) CometProject -Input [3]: [d_date_sk#17, d_year#39, d_moy#40] +(59) CometProject +Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(59) ColumnarToRow [codegen id : 1] +(60) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] -(60) BroadcastExchange +(61) BroadcastExchange Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (65) -+- * ColumnarToRow (64) - +- CometProject (63) - +- CometFilter (62) - +- CometScan parquet spark_catalog.default.date_dim (61) +Subquery:3 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 +BroadcastExchange (66) ++- * ColumnarToRow (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan parquet spark_catalog.default.date_dim (62) -(61) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#29, d_month_seq#41] +(62) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#31, d_month_seq#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(62) CometFilter -Input [2]: [d_date_sk#29, d_month_seq#41] -Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= Subquery scalar-subquery#42, [id=#43])) AND (d_month_seq#41 <= Subquery scalar-subquery#44, [id=#45])) AND isnotnull(d_date_sk#29)) +(63) CometFilter +Input [2]: [d_date_sk#31, d_month_seq#41] +Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= Subquery scalar-subquery#42, [id=#43])) AND (d_month_seq#41 <= Subquery scalar-subquery#44, [id=#45])) AND isnotnull(d_date_sk#31)) -(63) CometProject -Input [2]: [d_date_sk#29, d_month_seq#41] -Arguments: [d_date_sk#29], [d_date_sk#29] +(64) CometProject +Input [2]: [d_date_sk#31, d_month_seq#41] +Arguments: [d_date_sk#31], [d_date_sk#31] -(64) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#29] +(65) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#31] -(65) BroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +(66) BroadcastExchange +Input [1]: [d_date_sk#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:4 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* HashAggregate (72) -+- Exchange (71) - +- * ColumnarToRow (70) - +- CometHashAggregate (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan parquet spark_catalog.default.date_dim (66) +Subquery:4 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#42, [id=#43] +* HashAggregate (73) ++- Exchange (72) + +- * ColumnarToRow (71) + +- CometHashAggregate (70) + +- CometProject (69) + +- CometFilter (68) + +- CometScan parquet spark_catalog.default.date_dim (67) -(66) Scan parquet spark_catalog.default.date_dim +(67) Scan parquet spark_catalog.default.date_dim Output [3]: [d_month_seq#46, d_year#47, d_moy#48] 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,12)] ReadSchema: struct -(67) CometFilter +(68) CometFilter Input [3]: [d_month_seq#46, d_year#47, d_moy#48] Condition : (((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 12)) -(68) CometProject +(69) CometProject Input [3]: [d_month_seq#46, d_year#47, d_moy#48] Arguments: [(d_month_seq + 1)#49], [(d_month_seq#46 + 1) AS (d_month_seq + 1)#49] -(69) CometHashAggregate +(70) CometHashAggregate Input [1]: [(d_month_seq + 1)#49] Keys [1]: [(d_month_seq + 1)#49] Functions: [] -(70) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 1)#49] -(71) Exchange +(72) Exchange Input [1]: [(d_month_seq + 1)#49] -Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(72) HashAggregate [codegen id : 2] +(73) HashAggregate [codegen id : 2] Input [1]: [(d_month_seq + 1)#49] Keys [1]: [(d_month_seq + 1)#49] Functions: [] Aggregate Attributes: [] Results [1]: [(d_month_seq + 1)#49] -Subquery:5 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* HashAggregate (79) -+- Exchange (78) - +- * ColumnarToRow (77) - +- CometHashAggregate (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +Subquery:5 Hosting operator id = 63 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* HashAggregate (80) ++- Exchange (79) + +- * ColumnarToRow (78) + +- CometHashAggregate (77) + +- CometProject (76) + +- CometFilter (75) + +- CometScan parquet spark_catalog.default.date_dim (74) -(73) Scan parquet spark_catalog.default.date_dim +(74) Scan parquet spark_catalog.default.date_dim Output [3]: [d_month_seq#50, d_year#51, d_moy#52] 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,12)] ReadSchema: struct -(74) CometFilter +(75) CometFilter Input [3]: [d_month_seq#50, d_year#51, d_moy#52] Condition : (((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 1998)) AND (d_moy#52 = 12)) -(75) CometProject +(76) CometProject Input [3]: [d_month_seq#50, d_year#51, d_moy#52] Arguments: [(d_month_seq + 3)#53], [(d_month_seq#50 + 3) AS (d_month_seq + 3)#53] -(76) CometHashAggregate +(77) CometHashAggregate Input [1]: [(d_month_seq + 3)#53] Keys [1]: [(d_month_seq + 3)#53] Functions: [] -(77) ColumnarToRow [codegen id : 1] +(78) ColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 3)#53] -(78) Exchange +(79) Exchange Input [1]: [(d_month_seq + 3)#53] -Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(79) HashAggregate [codegen id : 2] +(80) HashAggregate [codegen id : 2] Input [1]: [(d_month_seq + 3)#53] Keys [1]: [(d_month_seq + 3)#53] Functions: [] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt index c1690facd4..c6886735f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (11) + WholeStageCodegen (8) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] InputAdapter Exchange [segment] #1 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [segment] [count,count] HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] InputAdapter Exchange [c_customer_sk] #2 - WholeStageCodegen (9) + WholeStageCodegen (6) HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] Project [c_customer_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -21,16 +21,16 @@ TakeOrderedAndProject [segment,num_customers,segment_base] HashAggregate [c_customer_sk,c_current_addr_sk] InputAdapter Exchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (4) - HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - BroadcastHashJoin [customer_sk,c_customer_sk] - Project [customer_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometProject [c_customer_sk,c_current_addr_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk] + CometProject [customer_sk] + CometBroadcastHashJoin [sold_date_sk,d_date_sk] + CometProject [sold_date_sk,customer_sk] + CometBroadcastHashJoin [item_sk,i_item_sk] CometUnion CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] CometFilter [cs_item_sk,cs_bill_customer_sk] @@ -47,32 +47,26 @@ TakeOrderedAndProject [segment,num_customers,segment_base] CometFilter [ws_item_sk,ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_category,i_class,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #5 + CometProject [i_item_sk] + CometFilter [i_category,i_class,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #7 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) + BroadcastExchange #8 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -82,7 +76,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) HashAggregate [(d_month_seq + 1)] InputAdapter - Exchange [(d_month_seq + 1)] #9 + Exchange [(d_month_seq + 1)] #10 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -94,7 +88,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (2) HashAggregate [(d_month_seq + 3)] InputAdapter - Exchange [(d_month_seq + 3)] #10 + Exchange [(d_month_seq + 3)] #11 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -104,18 +98,18 @@ TakeOrderedAndProject [segment,num_customers,segment_base] CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (6) + BroadcastExchange #12 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [ca_address_sk,ca_county,ca_state] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (7) + BroadcastExchange #13 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [s_county,s_state] CometScan parquet spark_catalog.default.store [s_county,s_state] InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [d_date_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt index 3d1d689bc9..00b14b96b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt @@ -1,25 +1,23 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * HashAggregate (20) - +- Exchange (19) - +- * HashAggregate (18) - +- * Project (17) - +- * BroadcastHashJoin Inner BuildRight (16) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.date_dim (1) - : +- BroadcastExchange (8) - : +- * ColumnarToRow (7) - : +- CometFilter (6) - : +- CometScan parquet spark_catalog.default.store_sales (5) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometProject (13) - +- CometFilter (12) - +- CometScan parquet spark_catalog.default.item (11) +TakeOrderedAndProject (19) ++- * HashAggregate (18) + +- Exchange (17) + +- * ColumnarToRow (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- CometBroadcastExchange (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.store_sales (4) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.item (9) (1) Scan parquet spark_catalog.default.date_dim @@ -37,10 +35,7 @@ Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) Input [3]: [d_date_sk#1, d_year#2, d_moy#3] Arguments: [d_date_sk#1], [d_date_sk#1] -(4) ColumnarToRow [codegen id : 3] -Input [1]: [d_date_sk#1] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -48,78 +43,71 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Condition : isnotnull(ss_item_sk#4) -(7) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [1]: [d_date_sk#1] +Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight -(10) Project [codegen id : 3] -Output [2]: [ss_item_sk#4, ss_ext_sales_price#5] +(8) CometProject Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: [ss_item_sk#4, ss_ext_sales_price#5], [ss_item_sk#4, ss_ext_sales_price#5] -(11) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] 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 -(12) CometFilter +(10) CometFilter Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) -(13) CometProject +(11) CometProject Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9], [i_item_sk#7, i_brand_id#8, i_brand#9] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9] -(15) BroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#4] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#4, ss_ext_sales_price#5] +Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight -(17) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +(14) CometProject Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(18) HashAggregate [codegen id : 3] +(15) CometHashAggregate Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] Keys [2]: [i_brand#9, i_brand_id#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#11] -Results [3]: [i_brand#9, i_brand_id#8, sum#12] -(19) Exchange -Input [3]: [i_brand#9, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(16) ColumnarToRow [codegen id : 1] +Input [3]: [i_brand#9, i_brand_id#8, sum#11] + +(17) Exchange +Input [3]: [i_brand#9, i_brand_id#8, sum#11] +Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] -Input [3]: [i_brand#9, i_brand_id#8, sum#12] +(18) HashAggregate [codegen id : 2] +Input [3]: [i_brand#9, i_brand_id#8, sum#11] Keys [2]: [i_brand#9, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [3]: [i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#12] +Results [3]: [i_brand_id#8 AS brand_id#13, i_brand#9 AS brand#14, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#12,17,2) AS ext_price#15] -(21) TakeOrderedAndProject -Input [3]: [brand_id#14, brand#15, ext_price#16] -Arguments: 100, [ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [brand_id#14, brand#15, ext_price#16] +(19) TakeOrderedAndProject +Input [3]: [brand_id#13, brand#14, ext_price#15] +Arguments: 100, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt index 7a0fe88633..2750a6ba20 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt @@ -1,31 +1,23 @@ TakeOrderedAndProject [ext_price,brand_id,brand] - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id] #1 - 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_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_brand,i_brand_id,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_brand_id,i_brand] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_sold_date_sk] CometProject [d_date_sk] CometFilter [d_moy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometBroadcastExchange #2 + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index bbed7eea67..74977244ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -6,62 +6,62 @@ TakeOrderedAndProject (63) +- Union (59) :- * HashAggregate (28) : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) + : +- * ColumnarToRow (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.customer_address (9) + : +- CometBroadcastExchange (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) : +- CometProject (19) : +- CometFilter (18) : +- CometScan parquet spark_catalog.default.item (17) :- * HashAggregate (43) : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * ColumnarToRow (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) + : +- * ColumnarToRow (41) + : +- CometHashAggregate (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) +- * HashAggregate (58) +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * ColumnarToRow (46) - : : : +- CometFilter (45) - : : : +- CometScan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) + +- * ColumnarToRow (56) + +- CometHashAggregate (55) + +- CometProject (54) + +- CometBroadcastHashJoin (53) + :- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometProject (48) + : : +- CometBroadcastHashJoin (47) + : : :- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet spark_catalog.default.store_sales @@ -76,199 +76,195 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Arguments: [d_date_sk#6], [d_date_sk#6] -(6) Project [codegen id : 5] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] + +(7) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -(7) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] +(9) Scan parquet spark_catalog.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 -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] +(10) CometFilter +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)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] +(11) CometProject +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] +Arguments: [ca_address_sk#9], [ca_address_sk#9] -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [ca_address_sk#9] +Arguments: [ca_address_sk#9] -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +Right output [1]: [ca_address_sk#9] +Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight -(13) Project [codegen id : 5] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] +(14) CometProject +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] +Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] -(14) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_item_id#10] +(15) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#11, i_item_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#9, i_item_id#10] +(16) CometFilter +Input [2]: [i_item_sk#11, i_item_id#12] +Condition : isnotnull(i_item_sk#11) (17) Scan parquet spark_catalog.default.item -Output [2]: [i_item_id#11, i_color#12] +Output [2]: [i_item_id#13, i_color#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [blanched ,burnished ,slate ])] ReadSchema: struct (18) CometFilter -Input [2]: [i_item_id#11, i_color#12] -Condition : i_color#12 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#13, i_color#14] +Condition : i_color#14 IN (slate ,blanched ,burnished ) (19) CometProject -Input [2]: [i_item_id#11, i_color#12] -Arguments: [i_item_id#11], [i_item_id#11] - -(20) ColumnarToRow [codegen id : 3] -Input [1]: [i_item_id#11] - -(21) BroadcastExchange -Input [1]: [i_item_id#11] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_id#10] -Right keys [1]: [i_item_id#11] -Join type: LeftSemi -Join condition: None - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_item_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#3, i_item_id#10] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#10] - -(26) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#3, i_item_id#10] -Keys [1]: [i_item_id#10] +Input [2]: [i_item_id#13, i_color#14] +Arguments: [i_item_id#13], [i_item_id#13] + +(20) CometBroadcastExchange +Input [1]: [i_item_id#13] +Arguments: [i_item_id#13] + +(21) CometBroadcastHashJoin +Left output [2]: [i_item_sk#11, i_item_id#12] +Right output [1]: [i_item_id#13] +Arguments: [i_item_id#12], [i_item_id#13], LeftSemi, BuildRight + +(22) CometBroadcastExchange +Input [2]: [i_item_sk#11, i_item_id#12] +Arguments: [i_item_sk#11, i_item_id#12] + +(23) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] +Right output [2]: [i_item_sk#11, i_item_id#12] +Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight + +(24) CometProject +Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#12] +Arguments: [ss_ext_sales_price#3, i_item_id#12], [ss_ext_sales_price#3, i_item_id#12] + +(25) CometHashAggregate +Input [2]: [ss_ext_sales_price#3, i_item_id#12] +Keys [1]: [i_item_id#12] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [i_item_id#10, sum#14] + +(26) ColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#12, sum#15] (27) Exchange -Input [2]: [i_item_id#10, sum#14] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [2]: [i_item_id#12, sum#15] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#10, sum#14] -Keys [1]: [i_item_id#10] +(28) HashAggregate [codegen id : 2] +Input [2]: [i_item_id#12, sum#15] +Keys [1]: [i_item_id#12] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_item_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] +Results [2]: [i_item_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS total_sales#17] (29) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (30) CometFilter -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) - -(31) ColumnarToRow [codegen id : 11] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] - -(32) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#22] - -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] - -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#23] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#23] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] - -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#24, i_item_id#25] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#24] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#19, i_item_id#25] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] - -(41) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#19, i_item_id#25] -Keys [1]: [i_item_id#25] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_item_id#25, sum#27] +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) + +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#23] + +(32) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Right output [1]: [d_date_sk#23] +Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight + +(33) CometProject +Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] +Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] + +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#24] + +(35) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] +Right output [1]: [ca_address_sk#24] +Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight + +(36) CometProject +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] +Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] + +(37) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#25, i_item_id#26] + +(38) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] +Right output [2]: [i_item_sk#25, i_item_id#26] +Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight + +(39) CometProject +Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] +Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] + +(40) CometHashAggregate +Input [2]: [cs_ext_sales_price#20, i_item_id#26] +Keys [1]: [i_item_id#26] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] + +(41) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_id#26, sum#27] (42) Exchange -Input [2]: [i_item_id#25, sum#27] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [2]: [i_item_id#26, sum#27] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#25, sum#27] -Keys [1]: [i_item_id#25] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] -Results [2]: [i_item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] +(43) HashAggregate [codegen id : 4] +Input [2]: [i_item_id#26, sum#27] +Keys [1]: [i_item_id#26] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#28] +Results [2]: [i_item_id#26, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#28,17,2) AS total_sales#29] (44) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] @@ -282,89 +278,84 @@ ReadSchema: struct (65) CometFilter -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2001)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#6)) +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) (66) CometProject -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] (67) ColumnarToRow [codegen id : 1] @@ -396,9 +387,9 @@ Input [1]: [d_date_sk#6] (68) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index f781ed1f7b..7fdead831d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (20) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (19) + WholeStageCodegen (7) HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #2 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,75 +31,59 @@ TakeOrderedAndProject [total_sales,i_item_id] CometProject [d_date_sk] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_item_id] - CometFilter [i_color] - CometScan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (12) + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange #6 + CometBroadcastHashJoin [i_item_id,i_item_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #7 + CometProject [i_item_id] + CometFilter [i_color] + CometScan parquet spark_catalog.default.item [i_item_id,i_color] + WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + Exchange [i_item_id] #9 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index bfd3495048..197603ca3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -13,23 +13,23 @@ TakeOrderedAndProject (45) : : +- Exchange (22) : : +- * HashAggregate (21) : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) + : : +- * ColumnarToRow (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.call_center (13) : +- BroadcastExchange (35) @@ -57,10 +57,7 @@ ReadSchema: struct Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] - -(4) Scan parquet spark_catalog.default.catalog_sales +(3) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -68,39 +65,46 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#7), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] Condition : (isnotnull(cs_item_sk#5) AND isnotnull(cs_call_center_sk#4)) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +Arguments: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Right output [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Arguments: [i_item_sk#1], [cs_item_sk#5], Inner, BuildRight -(9) Project [codegen id : 4] -Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] +(7) CometProject Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Arguments: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7], [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_year#10, d_moy#11] + +(11) CometBroadcastHashJoin +Left output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] +Right output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [cs_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(12) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] +(12) CometProject Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] +Arguments: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11], [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] (13) Scan parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#12, cc_name#13] @@ -113,145 +117,142 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#12, cc_name#13] Condition : (isnotnull(cc_call_center_sk#12) AND isnotnull(cc_name#13)) -(15) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [2]: [cc_call_center_sk#12, cc_name#13] +Arguments: [cc_call_center_sk#12, cc_name#13] -(16) BroadcastExchange -Input [2]: [cc_call_center_sk#12, cc_name#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] +Right output [2]: [cc_call_center_sk#12, cc_name#13] +Arguments: [cs_call_center_sk#4], [cc_call_center_sk#12], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#4] -Right keys [1]: [cc_call_center_sk#12] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] +(17) CometProject Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11, cc_call_center_sk#12, cc_name#13] +Arguments: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13], [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] -(19) HashAggregate [codegen id : 4] +(18) CometHashAggregate Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [1]: [sum#14] -Results [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] + +(19) ColumnarToRow [codegen id : 1] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#14] (20) Exchange -Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#14] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] +(21) HashAggregate [codegen id : 2] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#14] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#16] -Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS _w0#18] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#15] +Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#15,17,2) AS sum_sales#16, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#15,17,2) AS _w0#17] (22) Exchange -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(23) Sort [codegen id : 6] -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +(23) Sort [codegen id : 3] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 (24) Window -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#18], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +(25) Filter [codegen id : 4] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) (26) Window -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] +Arguments: [avg(_w0#17) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#19], [i_category#3, i_brand#2, cc_name#13, d_year#10] -(27) Filter [codegen id : 22] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) +(27) Filter [codegen id : 13] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] +Condition : ((isnotnull(avg_monthly_sales#19) AND (avg_monthly_sales#19 > 0.000000)) AND CASE WHEN (avg_monthly_sales#19 > 0.000000) THEN ((abs((sum_sales#16 - avg_monthly_sales#19)) / avg_monthly_sales#19) > 0.1000000000000000) END) -(28) Project [codegen id : 22] -Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +(28) Project [codegen id : 13] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] (29) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] +Output [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] -(30) HashAggregate [codegen id : 12] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] -Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] +(30) HashAggregate [codegen id : 6] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] +Keys [5]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24] +Functions [1]: [sum(UnscaledValue(cs_sales_price#26))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#26))#15] +Results [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, MakeDecimal(sum(UnscaledValue(cs_sales_price#26))#15,17,2) AS sum_sales#16] (31) Exchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: hashpartitioning(i_category#20, i_brand#21, cc_name#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 13] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 +(32) Sort [codegen id : 7] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, cc_name#22 ASC NULLS FIRST, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST], false, 0 (33) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [rank(d_year#23, d_moy#24) windowspecdefinition(i_category#20, i_brand#21, cc_name#22, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#27], [i_category#20, i_brand#21, cc_name#22], [d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] -(34) Project [codegen id : 14] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] +(34) Project [codegen id : 8] +Output [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#16 AS sum_sales#28, rn#27] +Input [7]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16, rn#27] (35) BroadcastExchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] +Input [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#28, rn#27] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] +(36) BroadcastHashJoin [codegen id : 13] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] +Right keys [4]: [i_category#20, i_brand#21, cc_name#22, (rn#27 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +(37) Project [codegen id : 13] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#28] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, cc_name#22, sum_sales#28, rn#27] (38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] -(39) Sort [codegen id : 20] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 +(39) Sort [codegen id : 11] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (40) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(41) Project [codegen id : 21] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] +(41) Project [codegen id : 12] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#16 AS sum_sales#35, rn#34] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16, rn#34] (42) BroadcastExchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#35, rn#34] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] +(43) BroadcastHashJoin [codegen id : 13] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#34 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +(44) Project [codegen id : 13] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#28 AS psum#36, sum_sales#35 AS nsum#37] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#28, i_category#29, i_brand#30, cc_name#31, sum_sales#35, rn#34] (45) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#36, nsum#37] +Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#36, nsum#37] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (49) +- * ColumnarToRow (48) +- CometFilter (47) @@ -274,6 +275,6 @@ Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index 3bc01343ae..c630cad48a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,74 +8,67 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter Exchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,cs_item_sk] CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange #3 + CometFilter [cs_item_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) + BroadcastExchange #7 + WholeStageCodegen (8) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) + WholeStageCodegen (7) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) + Exchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (6) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] InputAdapter ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #9 + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) + WholeStageCodegen (11) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index 7dc3730a8f..9d5fe57a27 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -1,53 +1,57 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- * BroadcastHashJoin Inner BuildRight (47) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Filter (16) - : : +- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (10) - : +- BroadcastExchange (30) - : +- * Filter (29) - : +- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometFilter (18) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- BroadcastExchange (46) - +- * Filter (45) - +- * HashAggregate (44) - +- Exchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * ColumnarToRow (35) - : : +- CometFilter (34) - : : +- CometScan parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (36) - +- ReusedExchange (39) +TakeOrderedAndProject (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Filter (20) + : : +- * HashAggregate (19) + : : +- Exchange (18) + : : +- * ColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometProject (15) + : : +- CometBroadcastHashJoin (14) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.item (3) + : : +- CometBroadcastExchange (13) + : : +- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- ReusedExchange (10) + : +- BroadcastExchange (34) + : +- * Filter (33) + : +- * HashAggregate (32) + : +- Exchange (31) + : +- * ColumnarToRow (30) + : +- CometHashAggregate (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) + : : +- ReusedExchange (23) + : +- ReusedExchange (26) + +- BroadcastExchange (50) + +- * Filter (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * ColumnarToRow (46) + +- CometHashAggregate (45) + +- CometProject (44) + +- CometBroadcastHashJoin (43) + :- CometProject (41) + : +- CometBroadcastHashJoin (40) + : :- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (39) + +- ReusedExchange (42) (1) Scan parquet spark_catalog.default.store_sales @@ -62,150 +66,163 @@ ReadSchema: struct Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#5, i_item_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [i_item_sk#5, i_item_id#6] Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) -(6) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#6] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#5, i_item_id#6] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Right output [2]: [i_item_sk#5, i_item_id#6] +Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) Project [codegen id : 4] -Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] +(7) CometProject Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6] +Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] + +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_date#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct -(10) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#7] +(9) CometFilter +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(10) ReusedExchange [Reuses operator id: 59] +Output [1]: [d_date#9] + +(11) CometBroadcastHashJoin +Left output [2]: [d_date_sk#7, d_date#8] +Right output [1]: [d_date#9] +Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight + +(12) CometProject +Input [2]: [d_date_sk#7, d_date#8] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(12) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#2, i_item_id#6] +(14) CometBroadcastHashJoin +Left output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#3], [d_date_sk#7], Inner, BuildRight + +(15) CometProject Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, d_date_sk#7] +Arguments: [ss_ext_sales_price#2, i_item_id#6], [ss_ext_sales_price#2, i_item_id#6] -(13) HashAggregate [codegen id : 4] +(16) CometHashAggregate Input [2]: [ss_ext_sales_price#2, i_item_id#6] Keys [1]: [i_item_id#6] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#8] -Results [2]: [i_item_id#6, sum#9] -(14) Exchange -Input [2]: [i_item_id#6, sum#9] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(17) ColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#6, sum#10] + +(18) Exchange +Input [2]: [i_item_id#6, sum#10] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#6, sum#9] +(19) HashAggregate [codegen id : 6] +Input [2]: [i_item_id#6, sum#10] Keys [1]: [i_item_id#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#10] -Results [2]: [i_item_id#6 AS item_id#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#10,17,2) AS ss_item_rev#12] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#11] +Results [2]: [i_item_id#6 AS item_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS ss_item_rev#13] -(16) Filter [codegen id : 15] -Input [2]: [item_id#11, ss_item_rev#12] -Condition : isnotnull(ss_item_rev#12) +(20) Filter [codegen id : 6] +Input [2]: [item_id#12, ss_item_rev#13] +Condition : isnotnull(ss_item_rev#13) -(17) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +(21) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(18) CometFilter -Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] -Condition : isnotnull(cs_item_sk#13) +(22) CometFilter +Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#14) -(19) ColumnarToRow [codegen id : 8] -Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +(23) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#18, i_item_id#19] -(20) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#17, i_item_id#18] +(24) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] +Right output [2]: [i_item_sk#18, i_item_id#19] +Arguments: [cs_item_sk#14], [i_item_sk#18], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#13] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(25) CometProject +Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] +Arguments: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19], [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] -(22) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18] -Input [5]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_sk#17, i_item_id#18] +(26) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#20] -(23) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#19] +(27) CometBroadcastHashJoin +Left output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] +Right output [1]: [d_date_sk#20] +Arguments: [cs_sold_date_sk#16], [d_date_sk#20], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#15] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#14, i_item_id#18] -Input [4]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18, d_date_sk#19] - -(26) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#14, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#14))] -Aggregate Attributes [1]: [sum#20] -Results [2]: [i_item_id#18, sum#21] - -(27) Exchange -Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(28) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#18, sum#21] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#14))#22] -Results [2]: [i_item_id#18 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#14))#22,17,2) AS cs_item_rev#24] - -(29) Filter [codegen id : 9] +(28) CometProject +Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] +Arguments: [cs_ext_sales_price#15, i_item_id#19], [cs_ext_sales_price#15, i_item_id#19] + +(29) CometHashAggregate +Input [2]: [cs_ext_sales_price#15, i_item_id#19] +Keys [1]: [i_item_id#19] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] + +(30) ColumnarToRow [codegen id : 2] +Input [2]: [i_item_id#19, sum#21] + +(31) Exchange +Input [2]: [i_item_id#19, sum#21] +Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(32) HashAggregate [codegen id : 3] +Input [2]: [i_item_id#19, sum#21] +Keys [1]: [i_item_id#19] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#15))#22] +Results [2]: [i_item_id#19 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#15))#22,17,2) AS cs_item_rev#24] + +(33) Filter [codegen id : 3] Input [2]: [item_id#23, cs_item_rev#24] Condition : isnotnull(cs_item_rev#24) -(30) BroadcastExchange +(34) BroadcastExchange Input [2]: [item_id#23, cs_item_rev#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(31) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#11] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [item_id#12] Right keys [1]: [item_id#23] Join type: Inner -Join condition: ((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) +Join condition: ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) -(32) Project [codegen id : 15] -Output [3]: [item_id#11, ss_item_rev#12, cs_item_rev#24] -Input [4]: [item_id#11, ss_item_rev#12, item_id#23, cs_item_rev#24] +(36) Project [codegen id : 6] +Output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#24] +Input [4]: [item_id#12, ss_item_rev#13, item_id#23, cs_item_rev#24] -(33) Scan parquet spark_catalog.default.web_sales +(37) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] @@ -213,172 +230,163 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(34) CometFilter +(38) CometFilter Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] Condition : isnotnull(ws_item_sk#25) -(35) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] - -(36) ReusedExchange [Reuses operator id: 7] +(39) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#29, i_item_id#30] -(37) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#25] -Right keys [1]: [i_item_sk#29] -Join type: Inner -Join condition: None +(40) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] +Right output [2]: [i_item_sk#29, i_item_id#30] +Arguments: [ws_item_sk#25], [i_item_sk#29], Inner, BuildRight -(38) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] +(41) CometProject Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#29, i_item_id#30] +Arguments: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30], [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] -(39) ReusedExchange [Reuses operator id: 60] +(42) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#31] -(40) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None +(43) CometBroadcastHashJoin +Left output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] +Right output [1]: [d_date_sk#31] +Arguments: [ws_sold_date_sk#27], [d_date_sk#31], Inner, BuildRight -(41) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#26, i_item_id#30] +(44) CometProject Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30, d_date_sk#31] +Arguments: [ws_ext_sales_price#26, i_item_id#30], [ws_ext_sales_price#26, i_item_id#30] -(42) HashAggregate [codegen id : 13] +(45) CometHashAggregate Input [2]: [ws_ext_sales_price#26, i_item_id#30] Keys [1]: [i_item_id#30] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#30, sum#33] -(43) Exchange -Input [2]: [i_item_id#30, sum#33] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(46) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_id#30, sum#32] + +(47) Exchange +Input [2]: [i_item_id#30, sum#32] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(44) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#30, sum#33] +(48) HashAggregate [codegen id : 5] +Input [2]: [i_item_id#30, sum#32] Keys [1]: [i_item_id#30] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#34] -Results [2]: [i_item_id#30 AS item_id#35, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#34,17,2) AS ws_item_rev#36] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#33] +Results [2]: [i_item_id#30 AS item_id#34, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#33,17,2) AS ws_item_rev#35] -(45) Filter [codegen id : 14] -Input [2]: [item_id#35, ws_item_rev#36] -Condition : isnotnull(ws_item_rev#36) +(49) Filter [codegen id : 5] +Input [2]: [item_id#34, ws_item_rev#35] +Condition : isnotnull(ws_item_rev#35) -(46) BroadcastExchange -Input [2]: [item_id#35, ws_item_rev#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] +(50) BroadcastExchange +Input [2]: [item_id#34, ws_item_rev#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [item_id#11] -Right keys [1]: [item_id#35] +(51) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [item_id#12] +Right keys [1]: [item_id#34] Join type: Inner -Join condition: ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * ws_item_rev#36)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) +Join condition: ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#35)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) -(48) Project [codegen id : 15] -Output [8]: [item_id#11, ss_item_rev#12, (((ss_item_rev#12 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ss_dev#37, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS cs_dev#38, ws_item_rev#36, (((ws_item_rev#36 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ws_dev#39, (((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36) / 3) AS average#40] -Input [5]: [item_id#11, ss_item_rev#12, cs_item_rev#24, item_id#35, ws_item_rev#36] +(52) Project [codegen id : 6] +Output [8]: [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ss_dev#36, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS cs_dev#37, ws_item_rev#35, (((ws_item_rev#35 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ws_dev#38, (((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35) / 3) AS average#39] +Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#24, item_id#34, ws_item_rev#35] -(49) TakeOrderedAndProject -Input [8]: [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] -Arguments: 100, [item_id#11 ASC NULLS FIRST, ss_item_rev#12 ASC NULLS FIRST], [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] +(53) TakeOrderedAndProject +Input [8]: [item_id#12, ss_item_rev#13, ss_dev#36, cs_item_rev#24, cs_dev#37, ws_item_rev#35, ws_dev#38, average#39] +Arguments: 100, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#36, cs_item_rev#24, cs_dev#37, ws_item_rev#35, ws_dev#38, average#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (60) -+- * Project (59) - +- * BroadcastHashJoin LeftSemi BuildRight (58) - :- * ColumnarToRow (52) - : +- CometFilter (51) - : +- CometScan parquet spark_catalog.default.date_dim (50) - +- BroadcastExchange (57) - +- * ColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan parquet spark_catalog.default.date_dim (53) - - -(50) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_date#41] +BroadcastExchange (63) ++- * ColumnarToRow (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometFilter (55) + : +- CometScan parquet spark_catalog.default.date_dim (54) + +- CometBroadcastExchange (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) + + +(54) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(51) CometFilter -Input [2]: [d_date_sk#7, d_date#41] +(55) CometFilter +Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(52) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#7, d_date#41] - -(53) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#42, d_week_seq#43] +(56) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#9, d_week_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct -(54) CometFilter -Input [2]: [d_date#42, d_week_seq#43] -Condition : (isnotnull(d_week_seq#43) AND (d_week_seq#43 = Subquery scalar-subquery#44, [id=#45])) +(57) CometFilter +Input [2]: [d_date#9, d_week_seq#40] +Condition : (isnotnull(d_week_seq#40) AND (d_week_seq#40 = Subquery scalar-subquery#41, [id=#42])) -(55) CometProject -Input [2]: [d_date#42, d_week_seq#43] -Arguments: [d_date#42], [d_date#42] +(58) CometProject +Input [2]: [d_date#9, d_week_seq#40] +Arguments: [d_date#9], [d_date#9] -(56) ColumnarToRow [codegen id : 1] -Input [1]: [d_date#42] +(59) CometBroadcastExchange +Input [1]: [d_date#9] +Arguments: [d_date#9] -(57) BroadcastExchange -Input [1]: [d_date#42] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=7] +(60) CometBroadcastHashJoin +Left output [2]: [d_date_sk#7, d_date#8] +Right output [1]: [d_date#9] +Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight -(58) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_date#41] -Right keys [1]: [d_date#42] -Join type: LeftSemi -Join condition: None +(61) CometProject +Input [2]: [d_date_sk#7, d_date#8] +Arguments: [d_date_sk#7], [d_date_sk#7] -(59) Project [codegen id : 2] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#41] +(62) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] -(60) BroadcastExchange +(63) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 54 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* ColumnarToRow (64) -+- CometProject (63) - +- CometFilter (62) - +- CometScan parquet spark_catalog.default.date_dim (61) +Subquery:2 Hosting operator id = 57 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +* ColumnarToRow (67) ++- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) -(61) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#46, d_week_seq#47] +(64) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#43, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(62) CometFilter -Input [2]: [d_date#46, d_week_seq#47] -Condition : (isnotnull(d_date#46) AND (d_date#46 = 2000-01-03)) +(65) CometFilter +Input [2]: [d_date#43, d_week_seq#44] +Condition : (isnotnull(d_date#43) AND (d_date#43 = 2000-01-03)) -(63) CometProject -Input [2]: [d_date#46, d_week_seq#47] -Arguments: [d_week_seq#47], [d_week_seq#47] +(66) CometProject +Input [2]: [d_date#43, d_week_seq#44] +Arguments: [d_week_seq#44], [d_week_seq#44] -(64) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#47] +(67) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#44] -Subquery:3 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt index d3bb0dc388..2ed2bde44b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (15) + WholeStageCodegen (6) Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] Project [item_id,ss_item_rev,cs_item_rev] @@ -8,90 +8,82 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (2) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date,d_date] CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date] - CometFilter [d_week_seq] - Subquery #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #3 + CometProject [d_date] + CometFilter [d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_date] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange #4 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date,d_date] + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (9) + BroadcastExchange #6 + WholeStageCodegen (3) Filter [cs_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] InputAdapter - Exchange [i_item_id] #6 - WholeStageCodegen (8) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #4 + ReusedExchange [d_date_sk] #5 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (14) + BroadcastExchange #8 + WholeStageCodegen (5) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (13) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [i_item_id] #9 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #4 + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt index 9bfb087f71..3af7e72960 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt @@ -1,48 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * HashAggregate (12) - : : : +- Exchange (11) - : : : +- * HashAggregate (10) - : : : +- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.date_dim (4) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.date_dim (19) - +- BroadcastExchange (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * HashAggregate (27) - : : +- ReusedExchange (26) - : +- BroadcastExchange (31) - : +- * ColumnarToRow (30) - : +- CometFilter (29) - : +- CometScan parquet spark_catalog.default.store (28) - +- BroadcastExchange (38) - +- * ColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan parquet spark_catalog.default.date_dim (34) +TakeOrderedAndProject (43) ++- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * HashAggregate (11) + : : : +- Exchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometHashAggregate (8) + : : : +- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (15) + : : +- * ColumnarToRow (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.store (12) + : +- BroadcastExchange (22) + : +- * ColumnarToRow (21) + : +- CometProject (20) + : +- CometFilter (19) + : +- CometScan parquet spark_catalog.default.date_dim (18) + +- BroadcastExchange (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * HashAggregate (26) + : : +- ReusedExchange (25) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.store (27) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) (1) Scan parquet spark_catalog.default.store_sales @@ -57,200 +56,194 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) ColumnarToRow [codegen id : 2] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.date_dim +(3) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#6] -(7) BroadcastExchange -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Right output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 2] -Output [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] +(7) CometProject Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#6] +Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] -(10) HashAggregate [codegen id : 2] +(8) CometHashAggregate Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] -Aggregate Attributes [7]: [sum#7, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13] -Results [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] -(11) Exchange -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(9) ColumnarToRow [codegen id : 1] +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#7, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13] + +(10) Exchange +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#7, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13] +Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(12) HashAggregate [codegen id : 10] -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] +(11) HashAggregate [codegen id : 8] +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#7, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27] -Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21,17,2) AS sun_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22,17,2) AS mon_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23,17,2) AS tue_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24,17,2) AS wed_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25,17,2) AS thu_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26,17,2) AS fri_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27,17,2) AS sat_sales#34] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#14, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#15, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#16, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#17, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#18, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#20] +Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#14,17,2) AS sun_sales#21, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#15,17,2) AS mon_sales#22, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#16,17,2) AS tue_sales#23, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#17,17,2) AS wed_sales#24, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#18,17,2) AS thu_sales#25, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#19,17,2) AS fri_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#20,17,2) AS sat_sales#27] -(13) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] +(12) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(14) CometFilter -Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] -Condition : (isnotnull(s_store_sk#35) AND isnotnull(s_store_id#36)) +(13) CometFilter +Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] +Condition : (isnotnull(s_store_sk#28) AND isnotnull(s_store_id#29)) -(15) ColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] +(14) ColumnarToRow [codegen id : 2] +Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] -(16) BroadcastExchange -Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(15) BroadcastExchange +Input [3]: [s_store_sk#28, s_store_id#29, s_store_name#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 10] +(16) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#35] +Right keys [1]: [s_store_sk#28] Join type: Inner Join condition: None -(18) Project [codegen id : 10] -Output [10]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37] -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_sk#35, s_store_id#36, s_store_name#37] +(17) Project [codegen id : 8] +Output [10]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#29, s_store_name#30] +Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_sk#28, s_store_id#29, s_store_name#30] -(19) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#38, d_week_seq#39] +(18) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#31, d_week_seq#32] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] ReadSchema: struct -(20) CometFilter -Input [2]: [d_month_seq#38, d_week_seq#39] -Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= 1212)) AND (d_month_seq#38 <= 1223)) AND isnotnull(d_week_seq#39)) +(19) CometFilter +Input [2]: [d_month_seq#31, d_week_seq#32] +Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1212)) AND (d_month_seq#31 <= 1223)) AND isnotnull(d_week_seq#32)) -(21) CometProject -Input [2]: [d_month_seq#38, d_week_seq#39] -Arguments: [d_week_seq#39], [d_week_seq#39] +(20) CometProject +Input [2]: [d_month_seq#31, d_week_seq#32] +Arguments: [d_week_seq#32], [d_week_seq#32] -(22) ColumnarToRow [codegen id : 4] -Input [1]: [d_week_seq#39] +(21) ColumnarToRow [codegen id : 3] +Input [1]: [d_week_seq#32] -(23) BroadcastExchange -Input [1]: [d_week_seq#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(22) BroadcastExchange +Input [1]: [d_week_seq#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 10] +(23) BroadcastHashJoin [codegen id : 8] Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#39] +Right keys [1]: [d_week_seq#32] Join type: Inner Join condition: None -(25) Project [codegen id : 10] -Output [10]: [s_store_name#37 AS s_store_name1#40, d_week_seq#5 AS d_week_seq1#41, s_store_id#36 AS s_store_id1#42, sun_sales#28 AS sun_sales1#43, mon_sales#29 AS mon_sales1#44, tue_sales#30 AS tue_sales1#45, wed_sales#31 AS wed_sales1#46, thu_sales#32 AS thu_sales1#47, fri_sales#33 AS fri_sales1#48, sat_sales#34 AS sat_sales1#49] -Input [11]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37, d_week_seq#39] +(24) Project [codegen id : 8] +Output [10]: [s_store_name#30 AS s_store_name1#33, d_week_seq#5 AS d_week_seq1#34, s_store_id#29 AS s_store_id1#35, sun_sales#21 AS sun_sales1#36, mon_sales#22 AS mon_sales1#37, tue_sales#23 AS tue_sales1#38, wed_sales#24 AS wed_sales1#39, thu_sales#25 AS thu_sales1#40, fri_sales#26 AS fri_sales1#41, sat_sales#27 AS sat_sales1#42] +Input [11]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#29, s_store_name#30, d_week_seq#32] -(26) ReusedExchange [Reuses operator id: 11] -Output [9]: [d_week_seq#5, ss_store_sk#1, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] +(25) ReusedExchange [Reuses operator id: 10] +Output [9]: [d_week_seq#5, ss_store_sk#1, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] -(27) HashAggregate [codegen id : 9] -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56] +(26) HashAggregate [codegen id : 7] +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#43, sum#44, sum#45, sum#46, sum#47, sum#48, sum#49] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27] -Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21,17,2) AS sun_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22,17,2) AS mon_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23,17,2) AS tue_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24,17,2) AS wed_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25,17,2) AS thu_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26,17,2) AS fri_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27,17,2) AS sat_sales#34] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#14, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#15, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#16, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#17, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#18, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#20] +Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#14,17,2) AS sun_sales#21, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#15,17,2) AS mon_sales#22, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#16,17,2) AS tue_sales#23, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#17,17,2) AS wed_sales#24, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#18,17,2) AS thu_sales#25, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#19,17,2) AS fri_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#20,17,2) AS sat_sales#27] -(28) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#57, s_store_id#58] +(27) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#50, s_store_id#51] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] ReadSchema: struct -(29) CometFilter -Input [2]: [s_store_sk#57, s_store_id#58] -Condition : (isnotnull(s_store_sk#57) AND isnotnull(s_store_id#58)) +(28) CometFilter +Input [2]: [s_store_sk#50, s_store_id#51] +Condition : (isnotnull(s_store_sk#50) AND isnotnull(s_store_id#51)) -(30) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#57, s_store_id#58] +(29) ColumnarToRow [codegen id : 5] +Input [2]: [s_store_sk#50, s_store_id#51] -(31) BroadcastExchange -Input [2]: [s_store_sk#57, s_store_id#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(30) BroadcastExchange +Input [2]: [s_store_sk#50, s_store_id#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(31) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#57] +Right keys [1]: [s_store_sk#50] Join type: Inner Join condition: None -(33) Project [codegen id : 9] -Output [9]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#58] -Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_sk#57, s_store_id#58] +(32) Project [codegen id : 7] +Output [9]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#51] +Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_sk#50, s_store_id#51] -(34) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#59, d_week_seq#60] +(33) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#52, d_week_seq#53] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] ReadSchema: struct -(35) CometFilter -Input [2]: [d_month_seq#59, d_week_seq#60] -Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1224)) AND (d_month_seq#59 <= 1235)) AND isnotnull(d_week_seq#60)) +(34) CometFilter +Input [2]: [d_month_seq#52, d_week_seq#53] +Condition : (((isnotnull(d_month_seq#52) AND (d_month_seq#52 >= 1224)) AND (d_month_seq#52 <= 1235)) AND isnotnull(d_week_seq#53)) -(36) CometProject -Input [2]: [d_month_seq#59, d_week_seq#60] -Arguments: [d_week_seq#60], [d_week_seq#60] +(35) CometProject +Input [2]: [d_month_seq#52, d_week_seq#53] +Arguments: [d_week_seq#53], [d_week_seq#53] -(37) ColumnarToRow [codegen id : 8] -Input [1]: [d_week_seq#60] +(36) ColumnarToRow [codegen id : 6] +Input [1]: [d_week_seq#53] -(38) BroadcastExchange -Input [1]: [d_week_seq#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(37) BroadcastExchange +Input [1]: [d_week_seq#53] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 7] Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#60] +Right keys [1]: [d_week_seq#53] Join type: Inner Join condition: None -(40) Project [codegen id : 9] -Output [9]: [d_week_seq#5 AS d_week_seq2#61, s_store_id#58 AS s_store_id2#62, sun_sales#28 AS sun_sales2#63, mon_sales#29 AS mon_sales2#64, tue_sales#30 AS tue_sales2#65, wed_sales#31 AS wed_sales2#66, thu_sales#32 AS thu_sales2#67, fri_sales#33 AS fri_sales2#68, sat_sales#34 AS sat_sales2#69] -Input [10]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#58, d_week_seq#60] +(39) Project [codegen id : 7] +Output [9]: [d_week_seq#5 AS d_week_seq2#54, s_store_id#51 AS s_store_id2#55, sun_sales#21 AS sun_sales2#56, mon_sales#22 AS mon_sales2#57, tue_sales#23 AS tue_sales2#58, wed_sales#24 AS wed_sales2#59, thu_sales#25 AS thu_sales2#60, fri_sales#26 AS fri_sales2#61, sat_sales#27 AS sat_sales2#62] +Input [10]: [d_week_seq#5, sun_sales#21, mon_sales#22, tue_sales#23, wed_sales#24, thu_sales#25, fri_sales#26, sat_sales#27, s_store_id#51, d_week_seq#53] -(41) BroadcastExchange -Input [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=7] +(40) BroadcastExchange +Input [9]: [d_week_seq2#54, s_store_id2#55, sun_sales2#56, mon_sales2#57, tue_sales2#58, wed_sales2#59, thu_sales2#60, fri_sales2#61, sat_sales2#62] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=6] -(42) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [s_store_id1#42, d_week_seq1#41] -Right keys [2]: [s_store_id2#62, (d_week_seq2#61 - 52)] +(41) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [s_store_id1#35, d_week_seq1#34] +Right keys [2]: [s_store_id2#55, (d_week_seq2#54 - 52)] Join type: Inner Join condition: None -(43) Project [codegen id : 10] -Output [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1#43 / sun_sales2#63) AS (sun_sales1 / sun_sales2)#70, (mon_sales1#44 / mon_sales2#64) AS (mon_sales1 / mon_sales2)#71, (tue_sales1#45 / tue_sales2#65) AS (tue_sales1 / tue_sales2)#72, (wed_sales1#46 / wed_sales2#66) AS (wed_sales1 / wed_sales2)#73, (thu_sales1#47 / thu_sales2#67) AS (thu_sales1 / thu_sales2)#74, (fri_sales1#48 / fri_sales2#68) AS (fri_sales1 / fri_sales2)#75, (sat_sales1#49 / sat_sales2#69) AS (sat_sales1 / sat_sales2)#76] -Input [19]: [s_store_name1#40, d_week_seq1#41, s_store_id1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] +(42) Project [codegen id : 8] +Output [10]: [s_store_name1#33, s_store_id1#35, d_week_seq1#34, (sun_sales1#36 / sun_sales2#56) AS (sun_sales1 / sun_sales2)#63, (mon_sales1#37 / mon_sales2#57) AS (mon_sales1 / mon_sales2)#64, (tue_sales1#38 / tue_sales2#58) AS (tue_sales1 / tue_sales2)#65, (wed_sales1#39 / wed_sales2#59) AS (wed_sales1 / wed_sales2)#66, (thu_sales1#40 / thu_sales2#60) AS (thu_sales1 / thu_sales2)#67, (fri_sales1#41 / fri_sales2#61) AS (fri_sales1 / fri_sales2)#68, (sat_sales1#42 / sat_sales2#62) AS (sat_sales1 / sat_sales2)#69] +Input [19]: [s_store_name1#33, d_week_seq1#34, s_store_id1#35, sun_sales1#36, mon_sales1#37, tue_sales1#38, wed_sales1#39, thu_sales1#40, fri_sales1#41, sat_sales1#42, d_week_seq2#54, s_store_id2#55, sun_sales2#56, mon_sales2#57, tue_sales2#58, wed_sales2#59, thu_sales2#60, fri_sales2#61, sat_sales2#62] -(44) TakeOrderedAndProject -Input [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] -Arguments: 100, [s_store_name1#40 ASC NULLS FIRST, s_store_id1#42 ASC NULLS FIRST, d_week_seq1#41 ASC NULLS FIRST], [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] +(43) TakeOrderedAndProject +Input [10]: [s_store_name1#33, s_store_id1#35, d_week_seq1#34, (sun_sales1 / sun_sales2)#63, (mon_sales1 / mon_sales2)#64, (tue_sales1 / tue_sales2)#65, (wed_sales1 / wed_sales2)#66, (thu_sales1 / thu_sales2)#67, (fri_sales1 / fri_sales2)#68, (sat_sales1 / sat_sales2)#69] +Arguments: 100, [s_store_name1#33 ASC NULLS FIRST, s_store_id1#35 ASC NULLS FIRST, d_week_seq1#34 ASC NULLS FIRST], [s_store_name1#33, s_store_id1#35, d_week_seq1#34, (sun_sales1 / sun_sales2)#63, (mon_sales1 / mon_sales2)#64, (tue_sales1 / tue_sales2)#65, (wed_sales1 / wed_sales2)#66, (thu_sales1 / thu_sales2)#67, (fri_sales1 / fri_sales2)#68, (sat_sales1 / sat_sales2)#69] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt index 9ad61e946e..e00d52dbbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - WholeStageCodegen (10) + WholeStageCodegen (8) Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] @@ -9,31 +9,27 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter Exchange [d_week_seq,ss_store_sk] #1 - WholeStageCodegen (2) - HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] + CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange #2 + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] InputAdapter BroadcastExchange #3 - WholeStageCodegen (3) + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [d_week_seq] @@ -41,7 +37,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] InputAdapter BroadcastExchange #5 - WholeStageCodegen (9) + WholeStageCodegen (7) Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] @@ -51,14 +47,14 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) + WholeStageCodegen (5) ColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_id] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) + WholeStageCodegen (6) ColumnarToRow InputAdapter CometProject [d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index 608154948c..95a19de484 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -1,43 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Filter (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer_address (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (29) - +- * Filter (28) - +- * HashAggregate (27) - +- Exchange (26) - +- * ColumnarToRow (25) - +- CometHashAggregate (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.item (22) +TakeOrderedAndProject (40) ++- * Filter (39) + +- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer_address (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.customer (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.store_sales (8) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.date_dim (13) + +- BroadcastExchange (33) + +- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- BroadcastExchange (30) + +- * Filter (29) + +- * HashAggregate (28) + +- Exchange (27) + +- * ColumnarToRow (26) + +- CometHashAggregate (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.item (23) (1) Scan parquet spark_catalog.default.customer_address @@ -51,38 +52,31 @@ ReadSchema: struct Input [2]: [ca_address_sk#1, ca_state#2] Condition : isnotnull(ca_address_sk#1) -(3) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#1, ca_state#2] - -(4) Scan parquet spark_catalog.default.customer +(3) Scan parquet spark_catalog.default.customer Output [2]: [c_customer_sk#3, c_current_addr_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Condition : (isnotnull(c_current_addr_sk#4) AND isnotnull(c_customer_sk#3)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [2]: [c_customer_sk#3, c_current_addr_sk#4] +Arguments: [c_customer_sk#3, c_current_addr_sk#4] -(7) BroadcastExchange -Input [2]: [c_customer_sk#3, c_current_addr_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [2]: [ca_address_sk#1, ca_state#2] +Right output [2]: [c_customer_sk#3, c_current_addr_sk#4] +Arguments: [ca_address_sk#1], [c_current_addr_sk#4], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ca_address_sk#1] -Right keys [1]: [c_current_addr_sk#4] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 7] -Output [2]: [ca_state#2, c_customer_sk#3] +(7) CometProject Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4] +Arguments: [ca_state#2, c_customer_sk#3], [ca_state#2, c_customer_sk#3] -(10) Scan parquet spark_catalog.default.store_sales +(8) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -90,216 +84,232 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Condition : (isnotnull(ss_customer_sk#6) AND isnotnull(ss_item_sk#5)) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -(13) BroadcastExchange -Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [2]: [ca_state#2, c_customer_sk#3] +Right output [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 7] -Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] +(12) CometProject Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7], [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] -(16) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#9] +(13) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_month_seq#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(14) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#10] +Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date_sk#9)) + +(15) CometProject +Input [2]: [d_date_sk#9, d_month_seq#10] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(16) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] + +(17) CometBroadcastHashJoin +Left output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(18) Project [codegen id : 7] -Output [2]: [ca_state#2, ss_item_sk#5] +(18) CometProject Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ca_state#2, ss_item_sk#5], [ca_state#2, ss_item_sk#5] + +(19) ColumnarToRow [codegen id : 4] +Input [2]: [ca_state#2, ss_item_sk#5] -(19) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#10, i_current_price#11, i_category#12] +(20) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#13, i_current_price#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] ReadSchema: struct -(20) CometFilter -Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] -Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10)) +(21) CometFilter +Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] +Condition : ((isnotnull(i_current_price#14) AND isnotnull(i_category#15)) AND isnotnull(i_item_sk#13)) -(21) ColumnarToRow [codegen id : 6] -Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] +(22) ColumnarToRow [codegen id : 3] +Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] -(22) Scan parquet spark_catalog.default.item -Output [2]: [i_current_price#13, i_category#14] +(23) Scan parquet spark_catalog.default.item +Output [2]: [i_current_price#16, i_category#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct -(23) CometFilter -Input [2]: [i_current_price#13, i_category#14] -Condition : isnotnull(i_category#14) +(24) CometFilter +Input [2]: [i_current_price#16, i_category#17] +Condition : isnotnull(i_category#17) -(24) CometHashAggregate -Input [2]: [i_current_price#13, i_category#14] -Keys [1]: [i_category#14] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] +(25) CometHashAggregate +Input [2]: [i_current_price#16, i_category#17] +Keys [1]: [i_category#17] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#16))] -(25) ColumnarToRow [codegen id : 4] -Input [3]: [i_category#14, sum#15, count#16] +(26) ColumnarToRow [codegen id : 1] +Input [3]: [i_category#17, sum#18, count#19] -(26) Exchange -Input [3]: [i_category#14, sum#15, count#16] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(27) Exchange +Input [3]: [i_category#17, sum#18, count#19] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(27) HashAggregate [codegen id : 5] -Input [3]: [i_category#14, sum#15, count#16] -Keys [1]: [i_category#14] -Functions [1]: [avg(UnscaledValue(i_current_price#13))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#17] -Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#17 / 100.0) as decimal(11,6)) AS avg(i_current_price)#18, i_category#14] +(28) HashAggregate [codegen id : 2] +Input [3]: [i_category#17, sum#18, count#19] +Keys [1]: [i_category#17] +Functions [1]: [avg(UnscaledValue(i_current_price#16))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#16))#20] +Results [2]: [cast((avg(UnscaledValue(i_current_price#16))#20 / 100.0) as decimal(11,6)) AS avg(i_current_price)#21, i_category#17] -(28) Filter [codegen id : 5] -Input [2]: [avg(i_current_price)#18, i_category#14] -Condition : isnotnull(avg(i_current_price)#18) +(29) Filter [codegen id : 2] +Input [2]: [avg(i_current_price)#21, i_category#17] +Condition : isnotnull(avg(i_current_price)#21) -(29) BroadcastExchange -Input [2]: [avg(i_current_price)#18, i_category#14] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] +(30) BroadcastExchange +Input [2]: [avg(i_current_price)#21, i_category#17] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=2] -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_category#12] -Right keys [1]: [i_category#14] +(31) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_category#15] +Right keys [1]: [i_category#17] Join type: Inner -Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#18)) +Join condition: (cast(i_current_price#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#21)) -(31) Project [codegen id : 6] -Output [1]: [i_item_sk#10] -Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#18, i_category#14] +(32) Project [codegen id : 3] +Output [1]: [i_item_sk#13] +Input [5]: [i_item_sk#13, i_current_price#14, i_category#15, avg(i_current_price)#21, i_category#17] -(32) BroadcastExchange -Input [1]: [i_item_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) BroadcastExchange +Input [1]: [i_item_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 7] +(34) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#10] +Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(34) Project [codegen id : 7] +(35) Project [codegen id : 4] Output [1]: [ca_state#2] -Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#10] +Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#13] -(35) HashAggregate [codegen id : 7] +(36) HashAggregate [codegen id : 4] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#19] -Results [2]: [ca_state#2, count#20] +Aggregate Attributes [1]: [count#22] +Results [2]: [ca_state#2, count#23] -(36) Exchange -Input [2]: [ca_state#2, count#20] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(37) Exchange +Input [2]: [ca_state#2, count#23] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(37) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#20] +(38) HashAggregate [codegen id : 5] +Input [2]: [ca_state#2, count#23] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#21] -Results [2]: [ca_state#2 AS state#22, count(1)#21 AS cnt#23] +Aggregate Attributes [1]: [count(1)#24] +Results [2]: [ca_state#2 AS state#25, count(1)#24 AS cnt#26] -(38) Filter [codegen id : 8] -Input [2]: [state#22, cnt#23] -Condition : (cnt#23 >= 10) +(39) Filter [codegen id : 5] +Input [2]: [state#25, cnt#26] +Condition : (cnt#26 >= 10) -(39) TakeOrderedAndProject -Input [2]: [state#22, cnt#23] -Arguments: 100, [cnt#23 ASC NULLS FIRST], [state#22, cnt#23] +(40) TakeOrderedAndProject +Input [2]: [state#25, cnt#26] +Arguments: 100, [cnt#26 ASC NULLS FIRST], [state#25, cnt#26] ===== Subqueries ===== -Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +Subquery:1 Hosting operator id = 8 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) -(40) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#24] +(41) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_month_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#24] -Condition : ((isnotnull(d_month_seq#24) AND (d_month_seq#24 = Subquery scalar-subquery#25, [id=#26])) AND isnotnull(d_date_sk#9)) +(42) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#10] +Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date_sk#9)) -(42) CometProject -Input [2]: [d_date_sk#9, d_month_seq#24] +(43) CometProject +Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(43) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(44) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#25, [id=#26] -* HashAggregate (51) -+- Exchange (50) - +- * ColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* HashAggregate (52) ++- Exchange (51) + +- * ColumnarToRow (50) + +- CometHashAggregate (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(45) Scan parquet spark_catalog.default.date_dim +(46) Scan parquet spark_catalog.default.date_dim Output [3]: [d_month_seq#27, d_year#28, d_moy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(46) CometFilter +(47) CometFilter Input [3]: [d_month_seq#27, d_year#28, d_moy#29] Condition : (((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2000)) AND (d_moy#29 = 1)) -(47) CometProject +(48) CometProject Input [3]: [d_month_seq#27, d_year#28, d_moy#29] Arguments: [d_month_seq#27], [d_month_seq#27] -(48) CometHashAggregate +(49) CometHashAggregate Input [1]: [d_month_seq#27] Keys [1]: [d_month_seq#27] Functions: [] -(49) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_month_seq#27] -(50) Exchange +(51) Exchange Input [1]: [d_month_seq#27] -Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(51) HashAggregate [codegen id : 2] +(52) HashAggregate [codegen id : 2] Input [1]: [d_month_seq#27] Keys [1]: [d_month_seq#27] Functions: [] Aggregate Attributes: [] Results [1]: [d_month_seq#27] +Subquery:3 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index fc6d5d0127..89a080d856 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -1,62 +1,57 @@ TakeOrderedAndProject [cnt,state] - WholeStageCodegen (8) + WholeStageCodegen (5) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] InputAdapter Exchange [ca_state] #1 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (2) - HashAggregate [d_month_seq] - InputAdapter - Exchange [d_month_seq] #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #2 + CometFilter [c_current_addr_sk,c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #3 + CometFilter [ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (2) + HashAggregate [d_month_seq] + InputAdapter + Exchange [d_month_seq] #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (3) Project [i_item_sk] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow @@ -64,13 +59,13 @@ TakeOrderedAndProject [cnt,state] CometFilter [i_current_price,i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) + BroadcastExchange #8 + WholeStageCodegen (2) Filter [avg(i_current_price)] HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter - Exchange [i_category] #8 - WholeStageCodegen (4) + Exchange [i_category] #9 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometHashAggregate [i_category,i_current_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 78f4b27ac9..0f61456ddb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -6,62 +6,62 @@ TakeOrderedAndProject (63) +- Union (59) :- * HashAggregate (28) : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : :- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.item (14) - : +- BroadcastExchange (21) - : +- * ColumnarToRow (20) + : +- * ColumnarToRow (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.customer_address (9) + : +- CometBroadcastExchange (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) : +- CometProject (19) : +- CometFilter (18) : +- CometScan parquet spark_catalog.default.item (17) :- * HashAggregate (43) : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * ColumnarToRow (31) - : : : : +- CometFilter (30) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) + : +- * ColumnarToRow (41) + : +- CometHashAggregate (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) +- * HashAggregate (58) +- Exchange (57) - +- * HashAggregate (56) - +- * Project (55) - +- * BroadcastHashJoin Inner BuildRight (54) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * ColumnarToRow (46) - : : : +- CometFilter (45) - : : : +- CometScan parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) + +- * ColumnarToRow (56) + +- CometHashAggregate (55) + +- CometProject (54) + +- CometBroadcastHashJoin (53) + :- CometProject (51) + : +- CometBroadcastHashJoin (50) + : :- CometProject (48) + : : +- CometBroadcastHashJoin (47) + : : :- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet spark_catalog.default.store_sales @@ -76,199 +76,195 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Arguments: [d_date_sk#6], [d_date_sk#6] -(6) Project [codegen id : 5] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] + +(7) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -(7) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] +(9) Scan parquet spark_catalog.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 -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] +(10) CometFilter +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)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] +(11) CometProject +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] +Arguments: [ca_address_sk#9], [ca_address_sk#9] -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [ca_address_sk#9] +Arguments: [ca_address_sk#9] -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +Right output [1]: [ca_address_sk#9] +Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight -(13) Project [codegen id : 5] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] +(14) CometProject +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] +Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] -(14) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_item_id#10] +(15) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#11, i_item_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#9, i_item_id#10] +(16) CometFilter +Input [2]: [i_item_sk#11, i_item_id#12] +Condition : isnotnull(i_item_sk#11) (17) Scan parquet spark_catalog.default.item -Output [2]: [i_item_id#11, i_category#12] +Output [2]: [i_item_id#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (18) CometFilter -Input [2]: [i_item_id#11, i_category#12] -Condition : (isnotnull(i_category#12) AND (i_category#12 = Music )) +Input [2]: [i_item_id#13, i_category#14] +Condition : (isnotnull(i_category#14) AND (i_category#14 = Music )) (19) CometProject -Input [2]: [i_item_id#11, i_category#12] -Arguments: [i_item_id#11], [i_item_id#11] - -(20) ColumnarToRow [codegen id : 3] -Input [1]: [i_item_id#11] - -(21) BroadcastExchange -Input [1]: [i_item_id#11] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_id#10] -Right keys [1]: [i_item_id#11] -Join type: LeftSemi -Join condition: None - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_item_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#3, i_item_id#10] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#10] - -(26) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#3, i_item_id#10] -Keys [1]: [i_item_id#10] +Input [2]: [i_item_id#13, i_category#14] +Arguments: [i_item_id#13], [i_item_id#13] + +(20) CometBroadcastExchange +Input [1]: [i_item_id#13] +Arguments: [i_item_id#13] + +(21) CometBroadcastHashJoin +Left output [2]: [i_item_sk#11, i_item_id#12] +Right output [1]: [i_item_id#13] +Arguments: [i_item_id#12], [i_item_id#13], LeftSemi, BuildRight + +(22) CometBroadcastExchange +Input [2]: [i_item_sk#11, i_item_id#12] +Arguments: [i_item_sk#11, i_item_id#12] + +(23) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] +Right output [2]: [i_item_sk#11, i_item_id#12] +Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight + +(24) CometProject +Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#12] +Arguments: [ss_ext_sales_price#3, i_item_id#12], [ss_ext_sales_price#3, i_item_id#12] + +(25) CometHashAggregate +Input [2]: [ss_ext_sales_price#3, i_item_id#12] +Keys [1]: [i_item_id#12] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [i_item_id#10, sum#14] + +(26) ColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#12, sum#15] (27) Exchange -Input [2]: [i_item_id#10, sum#14] -Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [2]: [i_item_id#12, sum#15] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 6] -Input [2]: [i_item_id#10, sum#14] -Keys [1]: [i_item_id#10] +(28) HashAggregate [codegen id : 2] +Input [2]: [i_item_id#12, sum#15] +Keys [1]: [i_item_id#12] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_item_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] +Results [2]: [i_item_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS total_sales#17] (29) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (30) CometFilter -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) - -(31) ColumnarToRow [codegen id : 11] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] - -(32) ReusedExchange [Reuses operator id: 68] -Output [1]: [d_date_sk#22] - -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] - -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#23] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#23] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] - -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#24, i_item_id#25] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#24] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#19, i_item_id#25] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] - -(41) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#19, i_item_id#25] -Keys [1]: [i_item_id#25] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#26] -Results [2]: [i_item_id#25, sum#27] +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) + +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#23] + +(32) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Right output [1]: [d_date_sk#23] +Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight + +(33) CometProject +Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] +Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] + +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#24] + +(35) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] +Right output [1]: [ca_address_sk#24] +Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight + +(36) CometProject +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] +Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] + +(37) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#25, i_item_id#26] + +(38) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] +Right output [2]: [i_item_sk#25, i_item_id#26] +Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight + +(39) CometProject +Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] +Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] + +(40) CometHashAggregate +Input [2]: [cs_ext_sales_price#20, i_item_id#26] +Keys [1]: [i_item_id#26] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] + +(41) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_id#26, sum#27] (42) Exchange -Input [2]: [i_item_id#25, sum#27] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [2]: [i_item_id#26, sum#27] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#25, sum#27] -Keys [1]: [i_item_id#25] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] -Results [2]: [i_item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] +(43) HashAggregate [codegen id : 4] +Input [2]: [i_item_id#26, sum#27] +Keys [1]: [i_item_id#26] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#28] +Results [2]: [i_item_id#26, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#28,17,2) AS total_sales#29] (44) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] @@ -282,89 +278,84 @@ ReadSchema: struct (65) CometFilter -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 9)) AND isnotnull(d_date_sk#6)) +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) (66) CometProject -Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] (67) ColumnarToRow [codegen id : 1] @@ -396,9 +387,9 @@ Input [1]: [d_date_sk#6] (68) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index b010414a86..b76e7c9b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -1,26 +1,26 @@ TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (20) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (19) + WholeStageCodegen (7) HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] InputAdapter Exchange [i_item_id] #2 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_addr_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -31,75 +31,59 @@ TakeOrderedAndProject [i_item_id,total_sales] CometProject [d_date_sk] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - BroadcastHashJoin [i_item_id,i_item_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [i_item_id] - CometFilter [i_category] - CometScan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (12) + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange #6 + CometBroadcastHashJoin [i_item_id,i_item_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #7 + CometProject [i_item_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_item_id,i_category] + WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (11) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_bill_addr_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 - WholeStageCodegen (17) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - 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] - ColumnarToRow - InputAdapter + Exchange [i_item_id] #9 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_addr_sk,ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index 766362167e..1567198fec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -1,71 +1,69 @@ == Physical Plan == -* Project (67) -+- * BroadcastNestedLoopJoin Inner BuildRight (66) - :- * HashAggregate (43) - : +- Exchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (26) - : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : :- * Project (20) - : : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * ColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.store (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan parquet spark_catalog.default.promotion (11) - : : : : +- ReusedExchange (18) - : : : +- BroadcastExchange (24) - : : : +- * ColumnarToRow (23) - : : : +- CometFilter (22) - : : : +- CometScan parquet spark_catalog.default.customer (21) - : : +- BroadcastExchange (31) - : : +- * ColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.customer_address (27) - : +- BroadcastExchange (38) - : +- * ColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan parquet spark_catalog.default.item (34) - +- BroadcastExchange (65) - +- * HashAggregate (64) - +- Exchange (63) - +- * HashAggregate (62) - +- * Project (61) - +- * BroadcastHashJoin Inner BuildRight (60) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * Project (55) - : : +- * BroadcastHashJoin Inner BuildRight (54) - : : :- * Project (52) - : : : +- * BroadcastHashJoin Inner BuildRight (51) - : : : :- * Project (49) - : : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : : :- * ColumnarToRow (46) - : : : : : +- CometFilter (45) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (44) - : : : : +- ReusedExchange (47) - : : : +- ReusedExchange (50) - : : +- ReusedExchange (53) - : +- ReusedExchange (56) - +- ReusedExchange (59) +* Project (65) ++- * BroadcastNestedLoopJoin Inner BuildRight (64) + :- * HashAggregate (41) + : +- Exchange (40) + : +- * ColumnarToRow (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometProject (20) + : : : : +- CometBroadcastHashJoin (19) + : : : : :- CometProject (14) + : : : : : +- CometBroadcastHashJoin (13) + : : : : : :- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (6) + : : : : : : +- CometProject (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.store (3) + : : : : : +- CometBroadcastExchange (12) + : : : : : +- CometProject (11) + : : : : : +- CometFilter (10) + : : : : : +- CometScan parquet spark_catalog.default.promotion (9) + : : : : +- CometBroadcastExchange (18) + : : : : +- CometProject (17) + : : : : +- CometFilter (16) + : : : : +- CometScan parquet spark_catalog.default.date_dim (15) + : : : +- CometBroadcastExchange (23) + : : : +- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.customer (21) + : : +- CometBroadcastExchange (29) + : : +- CometProject (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.customer_address (26) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometFilter (33) + : +- CometScan parquet spark_catalog.default.item (32) + +- BroadcastExchange (63) + +- * HashAggregate (62) + +- Exchange (61) + +- * ColumnarToRow (60) + +- CometHashAggregate (59) + +- CometProject (58) + +- CometBroadcastHashJoin (57) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometProject (52) + : : +- CometBroadcastHashJoin (51) + : : :- CometProject (49) + : : : +- CometBroadcastHashJoin (48) + : : : :- CometProject (46) + : : : : +- CometBroadcastHashJoin (45) + : : : : :- CometFilter (43) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (42) + : : : : +- ReusedExchange (44) + : : : +- ReusedExchange (47) + : : +- ReusedExchange (50) + : +- ReusedExchange (53) + +- ReusedExchange (56) (1) Scan parquet spark_catalog.default.store_sales @@ -80,338 +78,324 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [s_store_sk#8, s_gmt_offset#9] Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) -(6) CometProject +(5) CometProject Input [2]: [s_store_sk#8, s_gmt_offset#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(7) ColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#8] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [s_store_sk#8] -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Right output [1]: [s_store_sk#8] +Arguments: [ss_store_sk#3], [s_store_sk#8], Inner, BuildRight -(10) Project [codegen id : 7] -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +(8) CometProject Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(11) Scan parquet spark_catalog.default.promotion +(9) Scan parquet spark_catalog.default.promotion Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [Or(Or(EqualTo(p_channel_dmail,Y),EqualTo(p_channel_email,Y)),EqualTo(p_channel_tv,Y)), IsNotNull(p_promo_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] Condition : ((((p_channel_dmail#11 = Y) OR (p_channel_email#12 = Y)) OR (p_channel_tv#13 = Y)) AND isnotnull(p_promo_sk#10)) -(13) CometProject +(11) CometProject Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] Arguments: [p_promo_sk#10], [p_promo_sk#10] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [1]: [p_promo_sk#10] +Arguments: [p_promo_sk#10] -(15) BroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_promo_sk#4] -Right keys [1]: [p_promo_sk#10] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Right output [1]: [p_promo_sk#10] +Arguments: [ss_promo_sk#4], [p_promo_sk#10], Inner, BuildRight -(17) Project [codegen id : 7] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] +(14) CometProject Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -(18) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#14] +(15) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#14, d_year#15, d_moy#16] +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 -(19) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(16) CometFilter +Input [3]: [d_date_sk#14, d_year#15, d_moy#16] +Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) + +(17) CometProject +Input [3]: [d_date_sk#14, d_year#15, d_moy#16] +Arguments: [d_date_sk#14], [d_date_sk#14] -(20) Project [codegen id : 7] -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] +(18) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(19) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#6], [d_date_sk#14], Inner, BuildRight + +(20) CometProject Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] (21) Scan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#15, c_current_addr_sk#16] +Output [2]: [c_customer_sk#17, c_current_addr_sk#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (22) CometFilter -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) +Input [2]: [c_customer_sk#17, c_current_addr_sk#18] +Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) -(23) ColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] +(23) CometBroadcastExchange +Input [2]: [c_customer_sk#17, c_current_addr_sk#18] +Arguments: [c_customer_sk#17, c_current_addr_sk#18] -(24) BroadcastExchange -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None +(24) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] +Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] +Arguments: [ss_customer_sk#2], [c_customer_sk#17], Inner, BuildRight -(26) Project [codegen id : 7] -Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#15, c_current_addr_sk#16] +(25) CometProject +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#17, c_current_addr_sk#18] +Arguments: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18], [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] -(27) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_gmt_offset#18] +(26) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#19, ca_gmt_offset#20] 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 -(28) CometFilter -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -5.00)) AND isnotnull(ca_address_sk#17)) - -(29) CometProject -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Arguments: [ca_address_sk#17], [ca_address_sk#17] +(27) CometFilter +Input [2]: [ca_address_sk#19, ca_gmt_offset#20] +Condition : ((isnotnull(ca_gmt_offset#20) AND (ca_gmt_offset#20 = -5.00)) AND isnotnull(ca_address_sk#19)) -(30) ColumnarToRow [codegen id : 5] -Input [1]: [ca_address_sk#17] +(28) CometProject +Input [2]: [ca_address_sk#19, ca_gmt_offset#20] +Arguments: [ca_address_sk#19], [ca_address_sk#19] -(31) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(29) CometBroadcastExchange +Input [1]: [ca_address_sk#19] +Arguments: [ca_address_sk#19] -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#16] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] +Right output [1]: [ca_address_sk#19] +Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight -(33) Project [codegen id : 7] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16, ca_address_sk#17] +(31) CometProject +Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18, ca_address_sk#19] +Arguments: [ss_item_sk#1, ss_ext_sales_price#5], [ss_item_sk#1, ss_ext_sales_price#5] -(34) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_category#20] +(32) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#21, i_category#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Jewelry ), IsNotNull(i_item_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [i_item_sk#19, i_category#20] -Condition : ((isnotnull(i_category#20) AND (i_category#20 = Jewelry )) AND isnotnull(i_item_sk#19)) - -(36) CometProject -Input [2]: [i_item_sk#19, i_category#20] -Arguments: [i_item_sk#19], [i_item_sk#19] +(33) CometFilter +Input [2]: [i_item_sk#21, i_category#22] +Condition : ((isnotnull(i_category#22) AND (i_category#22 = Jewelry )) AND isnotnull(i_item_sk#21)) -(37) ColumnarToRow [codegen id : 6] -Input [1]: [i_item_sk#19] +(34) CometProject +Input [2]: [i_item_sk#21, i_category#22] +Arguments: [i_item_sk#21], [i_item_sk#21] -(38) BroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(35) CometBroadcastExchange +Input [1]: [i_item_sk#21] +Arguments: [i_item_sk#21] -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#1, ss_ext_sales_price#5] +Right output [1]: [i_item_sk#21] +Arguments: [ss_item_sk#1], [i_item_sk#21], Inner, BuildRight -(40) Project [codegen id : 7] -Output [1]: [ss_ext_sales_price#5] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#19] +(37) CometProject +Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#21] +Arguments: [ss_ext_sales_price#5], [ss_ext_sales_price#5] -(41) HashAggregate [codegen id : 7] +(38) CometHashAggregate Input [1]: [ss_ext_sales_price#5] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#21] -Results [1]: [sum#22] -(42) Exchange -Input [1]: [sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(39) ColumnarToRow [codegen id : 1] +Input [1]: [sum#23] + +(40) Exchange +Input [1]: [sum#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(43) HashAggregate [codegen id : 15] -Input [1]: [sum#22] +(41) HashAggregate [codegen id : 4] +Input [1]: [sum#23] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#23,17,2) AS promotions#24] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#24] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#24,17,2) AS promotions#25] -(44) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +(42) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#31)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(45) CometFilter -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) +(43) CometFilter +Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Condition : ((isnotnull(ss_store_sk#28) AND isnotnull(ss_customer_sk#27)) AND isnotnull(ss_item_sk#26)) -(46) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +(44) ReusedExchange [Reuses operator id: 6] +Output [1]: [s_store_sk#32] -(47) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#31] +(45) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Right output [1]: [s_store_sk#32] +Arguments: [ss_store_sk#28], [s_store_sk#32], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#27] -Right keys [1]: [s_store_sk#31] -Join type: Inner -Join condition: None +(46) CometProject +Input [6]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30, s_store_sk#32] +Arguments: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30], [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30] -(49) Project [codegen id : 13] -Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] +(47) ReusedExchange [Reuses operator id: 18] +Output [1]: [d_date_sk#33] -(50) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#32] +(48) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30] +Right output [1]: [d_date_sk#33] +Arguments: [ss_sold_date_sk#30], [d_date_sk#33], Inner, BuildRight -(51) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#32] -Join type: Inner -Join condition: None +(49) CometProject +Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30, d_date_sk#33] +Arguments: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29], [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29] -(52) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] +(50) ReusedExchange [Reuses operator id: 23] +Output [2]: [c_customer_sk#34, c_current_addr_sk#35] -(53) ReusedExchange [Reuses operator id: 24] -Output [2]: [c_customer_sk#33, c_current_addr_sk#34] +(51) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29] +Right output [2]: [c_customer_sk#34, c_current_addr_sk#35] +Arguments: [ss_customer_sk#27], [c_customer_sk#34], Inner, BuildRight -(54) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#33] -Join type: Inner -Join condition: None +(52) CometProject +Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, c_customer_sk#34, c_current_addr_sk#35] +Arguments: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35], [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35] -(55) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] +(53) ReusedExchange [Reuses operator id: 29] +Output [1]: [ca_address_sk#36] -(56) ReusedExchange [Reuses operator id: 31] -Output [1]: [ca_address_sk#35] +(54) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35] +Right output [1]: [ca_address_sk#36] +Arguments: [c_current_addr_sk#35], [ca_address_sk#36], Inner, BuildRight -(57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#35] -Join type: Inner -Join condition: None +(55) CometProject +Input [4]: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35, ca_address_sk#36] +Arguments: [ss_item_sk#26, ss_ext_sales_price#29], [ss_item_sk#26, ss_ext_sales_price#29] -(58) Project [codegen id : 13] -Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] +(56) ReusedExchange [Reuses operator id: 35] +Output [1]: [i_item_sk#37] -(59) ReusedExchange [Reuses operator id: 38] -Output [1]: [i_item_sk#36] +(57) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#26, ss_ext_sales_price#29] +Right output [1]: [i_item_sk#37] +Arguments: [ss_item_sk#26], [i_item_sk#37], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#25] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#28] -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] +(58) CometProject +Input [3]: [ss_item_sk#26, ss_ext_sales_price#29, i_item_sk#37] +Arguments: [ss_ext_sales_price#29], [ss_ext_sales_price#29] -(62) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#28] +(59) CometHashAggregate +Input [1]: [ss_ext_sales_price#29] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum#37] -Results [1]: [sum#38] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#29))] -(63) Exchange +(60) ColumnarToRow [codegen id : 2] Input [1]: [sum#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(64) HashAggregate [codegen id : 14] +(61) Exchange +Input [1]: [sum#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] + +(62) HashAggregate [codegen id : 3] Input [1]: [sum#38] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#28))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#39,17,2) AS total#40] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#29))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#29))#39] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#29))#39,17,2) AS total#40] -(65) BroadcastExchange +(63) BroadcastExchange Input [1]: [total#40] -Arguments: IdentityBroadcastMode, [plan_id=8] +Arguments: IdentityBroadcastMode, [plan_id=3] -(66) BroadcastNestedLoopJoin [codegen id : 15] +(64) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(67) Project [codegen id : 15] -Output [3]: [promotions#24, total#40, ((cast(promotions#24 as decimal(15,4)) / cast(total#40 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#41] -Input [2]: [promotions#24, total#40] +(65) Project [codegen id : 4] +Output [3]: [promotions#25, total#40, ((cast(promotions#25 as decimal(15,4)) / cast(total#40 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#41] +Input [2]: [promotions#25, total#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (70) ++- * ColumnarToRow (69) + +- CometProject (68) + +- CometFilter (67) + +- CometScan parquet spark_catalog.default.date_dim (66) -(68) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#42, d_moy#43] +(66) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#14, d_year#15, d_moy#16] 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 -(69) CometFilter -Input [3]: [d_date_sk#14, d_year#42, d_moy#43] -Condition : ((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 1998)) AND (d_moy#43 = 11)) AND isnotnull(d_date_sk#14)) +(67) CometFilter +Input [3]: [d_date_sk#14, d_year#15, d_moy#16] +Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) -(70) CometProject -Input [3]: [d_date_sk#14, d_year#42, d_moy#43] +(68) CometProject +Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(71) ColumnarToRow [codegen id : 1] +(69) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(72) BroadcastExchange +(70) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 44 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt index 2c3d07ac64..4ca1dd667a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt @@ -1,25 +1,25 @@ -WholeStageCodegen (15) +WholeStageCodegen (4) Project [promotions,total] BroadcastNestedLoopJoin HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] InputAdapter Exchange #1 - WholeStageCodegen (7) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_sales_price] + CometProject [ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -30,77 +30,54 @@ WholeStageCodegen (15) CometProject [d_date_sk] CometFilter [d_year,d_moy,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_gmt_offset,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometBroadcastExchange #3 + CometProject [s_store_sk] + CometFilter [s_gmt_offset,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] + CometBroadcastExchange #4 + CometProject [p_promo_sk] + CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #7 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange #8 + CometProject [i_item_sk] + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (14) + BroadcastExchange #9 + WholeStageCodegen (3) HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] InputAdapter - Exchange #9 - WholeStageCodegen (13) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + Exchange #10 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ext_sales_price] + CometProject [ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [s_store_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #6 - InputAdapter - ReusedExchange [i_item_sk] #7 + ReusedExchange [s_store_sk] #3 + ReusedExchange [d_date_sk] #5 + ReusedExchange [c_customer_sk,c_current_addr_sk] #6 + ReusedExchange [ca_address_sk] #7 + ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt index 0607d80770..21d44db2c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt @@ -1,36 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.ship_mode (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.web_site (16) - +- BroadcastExchange (26) - +- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +TakeOrderedAndProject (28) ++- * HashAggregate (27) + +- Exchange (26) + +- * ColumnarToRow (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometProject (17) + : +- CometBroadcastHashJoin (16) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.warehouse (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.ship_mode (8) + : +- CometBroadcastExchange (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.web_site (13) + +- CometBroadcastExchange (21) + +- CometProject (20) + +- CometFilter (19) + +- CometScan parquet spark_catalog.default.date_dim (18) (1) Scan parquet spark_catalog.default.web_sales @@ -44,144 +40,126 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Condition : isnotnull(w_warehouse_sk#6) -(6) ColumnarToRow [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [w_warehouse_sk#6, w_warehouse_name#7] -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#4] -Right keys [1]: [w_warehouse_sk#6] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] +Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [ws_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight -(9) Project [codegen id : 5] -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] +(7) CometProject Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] -(10) Scan parquet spark_catalog.default.ship_mode +(8) Scan parquet spark_catalog.default.ship_mode Output [2]: [sm_ship_mode_sk#8, sm_type#9] Batched: true Location [not included in comparison]/{warehouse_dir}/ship_mode] PushedFilters: [IsNotNull(sm_ship_mode_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [sm_ship_mode_sk#8, sm_type#9] Condition : isnotnull(sm_ship_mode_sk#8) -(12) ColumnarToRow [codegen id : 2] -Input [2]: [sm_ship_mode_sk#8, sm_type#9] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [sm_ship_mode_sk#8, sm_type#9] -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_mode_sk#3] -Right keys [1]: [sm_ship_mode_sk#8] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] +Right output [2]: [sm_ship_mode_sk#8, sm_type#9] +Arguments: [ws_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight -(15) Project [codegen id : 5] -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9] +(12) CometProject Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#9] +Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9] -(16) Scan parquet spark_catalog.default.web_site +(13) Scan parquet spark_catalog.default.web_site Output [2]: [web_site_sk#10, web_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [2]: [web_site_sk#10, web_name#11] Condition : isnotnull(web_site_sk#10) -(18) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [2]: [web_site_sk#10, web_name#11] +Arguments: [web_site_sk#10, web_name#11] -(19) BroadcastExchange -Input [2]: [web_site_sk#10, web_name#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_web_site_sk#2] -Right keys [1]: [web_site_sk#10] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9] +Right output [2]: [web_site_sk#10, web_name#11] +Arguments: [ws_web_site_sk#2], [web_site_sk#10], Inner, BuildRight -(21) Project [codegen id : 5] -Output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11] +(17) CometProject Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_site_sk#10, web_name#11] +Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11], [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11] -(22) Scan parquet spark_catalog.default.date_dim +(18) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_month_seq#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(23) CometFilter +(19) CometFilter Input [2]: [d_date_sk#12, d_month_seq#13] Condition : (((isnotnull(d_month_seq#13) AND (d_month_seq#13 >= 1200)) AND (d_month_seq#13 <= 1211)) AND isnotnull(d_date_sk#12)) -(24) CometProject +(20) CometProject Input [2]: [d_date_sk#12, d_month_seq#13] Arguments: [d_date_sk#12], [d_date_sk#12] -(25) ColumnarToRow [codegen id : 4] +(21) CometBroadcastExchange Input [1]: [d_date_sk#12] +Arguments: [d_date_sk#12] -(26) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(22) CometBroadcastHashJoin +Left output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11] +Right output [1]: [d_date_sk#12] +Arguments: [ws_ship_date_sk#1], [d_date_sk#12], Inner, BuildRight -(27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 5] -Output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] +(23) CometProject Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11, d_date_sk#12] +Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _groupingexpression#14], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] -(29) HashAggregate [codegen id : 5] +(24) CometHashAggregate Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _groupingexpression#14] Keys [3]: [_groupingexpression#14, sm_type#9, web_name#11] Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] -Results [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -(30) Exchange -Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, web_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(25) ColumnarToRow [codegen id : 1] +Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#15, sum#16, sum#17, sum#18, sum#19] + +(26) Exchange +Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#15, sum#16, sum#17, sum#18, sum#19] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, web_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(31) HashAggregate [codegen id : 6] -Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] +(27) HashAggregate [codegen id : 2] +Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#15, sum#16, sum#17, sum#18, sum#19] Keys [3]: [_groupingexpression#14, sm_type#9, web_name#11] Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] -Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] +Aggregate Attributes [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#20, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#21, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#22, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#23, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#24] +Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#25, sm_type#9, web_name#11, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#20 AS 30 days #26, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#21 AS 31 - 60 days #27, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#22 AS 61 - 90 days #28, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#23 AS 91 - 120 days #29, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#24 AS >120 days #30] -(32) TakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] -Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] +(28) TakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#25, sm_type#9, web_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] +Arguments: 100, [substr(w_warehouse_name, 1, 20)#25 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#25, sm_type#9, web_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt index 5ae522ce1c..c6b7e18345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt @@ -1,48 +1,32 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter Exchange [_groupingexpression,sm_type,web_name] #1 - WholeStageCodegen (5) - HashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,w_warehouse_name] - BroadcastHashJoin [ws_ship_date_sk,d_date_sk] - Project [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] - BroadcastHashJoin [ws_web_site_sk,web_site_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] + CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] + CometBroadcastHashJoin [ws_ship_date_sk,d_date_sk] + CometProject [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] + CometBroadcastHashJoin [ws_web_site_sk,web_site_sk] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #2 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #3 + CometFilter [sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange #4 + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index 990bc31957..621e9c8c64 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -1,32 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (28) -+- * Project (27) - +- * Filter (26) - +- Window (25) - +- * Sort (24) - +- Exchange (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * ColumnarToRow (16) - +- CometFilter (15) - +- CometScan parquet spark_catalog.default.store (14) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * Sort (25) + +- Exchange (24) + +- * HashAggregate (23) + +- Exchange (22) + +- * ColumnarToRow (21) + +- CometHashAggregate (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- CometBroadcastExchange (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.store_sales (4) + : +- CometBroadcastExchange (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.date_dim (9) + +- CometBroadcastExchange (17) + +- CometFilter (16) + +- CometScan parquet spark_catalog.default.store (15) (1) Scan parquet spark_catalog.default.item @@ -44,10 +45,7 @@ Condition : ((((i_category#4 IN (Books Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] -(4) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#1, i_manager_id#5] - -(5) Scan parquet spark_catalog.default.store_sales +(4) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] @@ -55,140 +53,148 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) -(7) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [2]: [i_item_sk#1, i_manager_id#5] +Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight -(10) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +(8) CometProject Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] + +(9) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] +ReadSchema: struct + +(10) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] +Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(11) ReusedExchange [Reuses operator id: 33] -Output [2]: [d_date_sk#15, d_moy#16] +(11) CometProject +Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] +Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(12) CometBroadcastExchange +Input [2]: [d_date_sk#15, d_moy#17] +Arguments: [d_date_sk#15, d_moy#17] -(13) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16] -Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#16] +(13) CometBroadcastHashJoin +Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Right output [2]: [d_date_sk#15, d_moy#17] +Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(14) Scan parquet spark_catalog.default.store -Output [1]: [s_store_sk#17] +(14) CometProject +Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#17] +Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] + +(15) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#18] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [1]: [s_store_sk#17] -Condition : isnotnull(s_store_sk#17) - -(16) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#17] +(16) CometFilter +Input [1]: [s_store_sk#18] +Condition : isnotnull(s_store_sk#18) -(17) BroadcastExchange -Input [1]: [s_store_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [1]: [s_store_sk#18] +Arguments: [s_store_sk#18] -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#17] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] +Right output [1]: [s_store_sk#18] +Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight -(19) Project [codegen id : 4] -Output [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] -Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16, s_store_sk#17] +(19) CometProject +Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17, s_store_sk#18] +Arguments: [i_manager_id#5, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_sales_price#12, d_moy#17] -(20) HashAggregate [codegen id : 4] -Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] -Keys [2]: [i_manager_id#5, d_moy#16] +(20) CometHashAggregate +Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#17] +Keys [2]: [i_manager_id#5, d_moy#17] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [i_manager_id#5, d_moy#16, sum#19] -(21) Exchange -Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) ColumnarToRow [codegen id : 1] +Input [3]: [i_manager_id#5, d_moy#17, sum#19] + +(22) Exchange +Input [3]: [i_manager_id#5, d_moy#17, sum#19] +Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] -Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Keys [2]: [i_manager_id#5, d_moy#16] +(23) HashAggregate [codegen id : 2] +Input [3]: [i_manager_id#5, d_moy#17, sum#19] +Keys [2]: [i_manager_id#5, d_moy#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] -(23) Exchange +(24) Exchange Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(24) Sort [codegen id : 6] +(25) Sort [codegen id : 3] Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [i_manager_id#5 ASC NULLS FIRST], false, 0 -(25) Window +(26) Window Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] -(26) Filter [codegen id : 7] +(27) Filter [codegen id : 4] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END -(27) Project [codegen id : 7] +(28) Project [codegen id : 4] Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -(28) TakeOrderedAndProject +(29) TakeOrderedAndProject Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (33) -+- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.date_dim (29) +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) -(29) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] +(30) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(30) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) +(31) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] +Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(31) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] +(32) CometProject +Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] +Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(32) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_moy#16] +(33) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#15, d_moy#17] -(33) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(34) BroadcastExchange +Input [2]: [d_date_sk#15, d_moy#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index 7f6f8c1370..35e09ec7dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -1,51 +1,45 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_manager_id] InputAdapter Exchange [i_manager_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_manager_id,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - Project [i_manager_id,ss_sales_price,d_moy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_manager_id,d_moy,ss_sales_price] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometProject [i_item_sk,i_manager_id] CometFilter [i_category,i_class,i_brand,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index 95cb8ab519..bd491e60f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -216,7 +216,7 @@ Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_numbe (7) CometBroadcastHashJoin Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner +Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft (8) CometProject Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] @@ -725,7 +725,7 @@ Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_nu (117) CometBroadcastHashJoin Left output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Right output [2]: [sr_item_sk#119, sr_ticket_number#120] -Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner +Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner, BuildLeft (118) CometProject Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index eda84bb525..a10b6897c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -1,43 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) +TakeOrderedAndProject (42) ++- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) : : :- * ColumnarToRow (3) : : : +- CometFilter (2) : : : +- CometScan parquet spark_catalog.default.store (1) - : : +- BroadcastExchange (14) - : : +- * Filter (13) - : : +- * HashAggregate (12) - : : +- Exchange (11) - : : +- * HashAggregate (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : +- ReusedExchange (7) - : +- BroadcastExchange (20) - : +- * ColumnarToRow (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.item (17) - +- BroadcastExchange (36) - +- * Filter (35) - +- * HashAggregate (34) - +- Exchange (33) - +- * HashAggregate (32) - +- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * ColumnarToRow (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.store_sales (23) - +- ReusedExchange (26) + : : +- BroadcastExchange (17) + : : +- * Filter (16) + : : +- * HashAggregate (15) + : : +- Exchange (14) + : : +- * ColumnarToRow (13) + : : +- CometHashAggregate (12) + : : +- CometProject (11) + : : +- CometBroadcastHashJoin (10) + : : :- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : +- CometBroadcastExchange (9) + : : +- CometProject (8) + : : +- CometFilter (7) + : : +- CometScan parquet spark_catalog.default.date_dim (6) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- BroadcastExchange (39) + +- * Filter (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * HashAggregate (34) + +- Exchange (33) + +- * ColumnarToRow (32) + +- CometHashAggregate (31) + +- CometProject (30) + +- CometBroadcastHashJoin (29) + :- CometFilter (27) + : +- CometScan parquet spark_catalog.default.store_sales (26) + +- ReusedExchange (28) (1) Scan parquet spark_catalog.default.store @@ -51,7 +54,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) ColumnarToRow [codegen id : 9] +(3) ColumnarToRow [codegen id : 7] Input [2]: [s_store_sk#1, s_store_name#2] (4) Scan parquet spark_catalog.default.store_sales @@ -66,87 +69,100 @@ ReadSchema: struct Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) -(6) ColumnarToRow [codegen id : 2] -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +(6) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#8, d_month_seq#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] +ReadSchema: struct -(7) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#8] +(7) CometFilter +Input [2]: [d_date_sk#8, d_month_seq#9] +Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] -Join type: Inner -Join condition: None +(8) CometProject +Input [2]: [d_date_sk#8, d_month_seq#9] +Arguments: [d_date_sk#8], [d_date_sk#8] + +(9) CometBroadcastExchange +Input [1]: [d_date_sk#8] +Arguments: [d_date_sk#8] -(9) Project [codegen id : 2] -Output [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] +(10) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +Right output [1]: [d_date_sk#8] +Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight + +(11) CometProject Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] +Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -(10) HashAggregate [codegen id : 2] +(12) CometHashAggregate Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum#9] -Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -(11) Exchange +(13) ColumnarToRow [codegen id : 1] +Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] + +(14) Exchange Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(12) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 2] Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] -(13) Filter [codegen id : 3] +(16) Filter [codegen id : 2] Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] Condition : isnotnull(revenue#12) -(14) BroadcastExchange +(17) BroadcastExchange Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 9] +(18) BroadcastHashJoin [codegen id : 7] Left keys [1]: [s_store_sk#1] Right keys [1]: [ss_store_sk#4] Join type: Inner Join condition: None -(16) Project [codegen id : 9] +(19) Project [codegen id : 7] Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] -(17) Scan parquet spark_catalog.default.item +(20) Scan parquet spark_catalog.default.item Output [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(18) CometFilter +(21) CometFilter Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Condition : isnotnull(i_item_sk#13) -(19) ColumnarToRow [codegen id : 4] +(22) ColumnarToRow [codegen id : 3] Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -(20) BroadcastExchange +(23) BroadcastExchange Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(21) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#3] Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(22) Project [codegen id : 9] +(25) Project [codegen id : 7] Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -(23) Scan parquet spark_catalog.default.store_sales +(26) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] @@ -154,116 +170,113 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(24) CometFilter +(27) CometFilter Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] Condition : isnotnull(ss_store_sk#19) -(25) ColumnarToRow [codegen id : 6] -Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] - -(26) ReusedExchange [Reuses operator id: 44] +(28) ReusedExchange [Reuses operator id: 9] Output [1]: [d_date_sk#23] -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#21] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None +(29) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +Right output [1]: [d_date_sk#23] +Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight -(28) Project [codegen id : 6] -Output [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] +(30) CometProject Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] +Arguments: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20], [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -(29) HashAggregate [codegen id : 6] +(31) CometHashAggregate Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] Keys [2]: [ss_store_sk#19, ss_item_sk#18] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] -(30) Exchange -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] +(32) ColumnarToRow [codegen id : 4] +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] + +(33) Exchange +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(31) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] +(34) HashAggregate [codegen id : 5] +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] Keys [2]: [ss_store_sk#19, ss_item_sk#18] Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#20))#26] -Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#26,17,2) AS revenue#27] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#20))#25] +Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#25,17,2) AS revenue#26] -(32) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#19, revenue#27] +(35) HashAggregate [codegen id : 5] +Input [2]: [ss_store_sk#19, revenue#26] Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#27)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ss_store_sk#19, sum#30, count#31] +Functions [1]: [partial_avg(revenue#26)] +Aggregate Attributes [2]: [sum#27, count#28] +Results [3]: [ss_store_sk#19, sum#29, count#30] -(33) Exchange -Input [3]: [ss_store_sk#19, sum#30, count#31] +(36) Exchange +Input [3]: [ss_store_sk#19, sum#29, count#30] Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(34) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#19, sum#30, count#31] +(37) HashAggregate [codegen id : 6] +Input [3]: [ss_store_sk#19, sum#29, count#30] Keys [1]: [ss_store_sk#19] -Functions [1]: [avg(revenue#27)] -Aggregate Attributes [1]: [avg(revenue#27)#32] -Results [2]: [ss_store_sk#19, avg(revenue#27)#32 AS ave#33] +Functions [1]: [avg(revenue#26)] +Aggregate Attributes [1]: [avg(revenue#26)#31] +Results [2]: [ss_store_sk#19, avg(revenue#26)#31 AS ave#32] -(35) Filter [codegen id : 8] -Input [2]: [ss_store_sk#19, ave#33] -Condition : isnotnull(ave#33) +(38) Filter [codegen id : 6] +Input [2]: [ss_store_sk#19, ave#32] +Condition : isnotnull(ave#32) -(36) BroadcastExchange -Input [2]: [ss_store_sk#19, ave#33] +(39) BroadcastExchange +Input [2]: [ss_store_sk#19, ave#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(37) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#19] Join type: Inner -Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) +Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#32)) -(38) Project [codegen id : 9] +(41) Project [codegen id : 7] Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17, ss_store_sk#19, ave#33] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17, ss_store_sk#19, ave#32] -(39) TakeOrderedAndProject +(42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(40) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#34] +(43) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#8, d_month_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_month_seq#34 <= 1187)) AND isnotnull(d_date_sk#8)) +(44) CometFilter +Input [2]: [d_date_sk#8, d_month_seq#9] +Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) -(42) CometProject -Input [2]: [d_date_sk#8, d_month_seq#34] +(45) CometProject +Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(43) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(44) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt index 33b695e811..8de564ed16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (9) + WholeStageCodegen (7) Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] @@ -12,17 +12,17 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [revenue] HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter Exchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -33,35 +33,36 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) + BroadcastExchange #5 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) + BroadcastExchange #6 + WholeStageCodegen (6) Filter [ave] HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] InputAdapter - Exchange [ss_store_sk] #6 - WholeStageCodegen (7) + Exchange [ss_store_sk] #7 + WholeStageCodegen (5) HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] InputAdapter - Exchange [ss_store_sk,ss_item_sk] #7 - WholeStageCodegen (6) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [ss_store_sk,ss_item_sk] #8 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index a26c457aa4..6bca75e27e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -1,56 +1,55 @@ == Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- Union (48) - :- * HashAggregate (29) - : +- Exchange (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.time_dim (13) - : +- BroadcastExchange (24) - : +- * ColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan parquet spark_catalog.default.ship_mode (20) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (32) - : : : : +- CometFilter (31) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- ReusedExchange (42) +TakeOrderedAndProject (51) ++- * HashAggregate (50) + +- Exchange (49) + +- * HashAggregate (48) + +- Union (47) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * ColumnarToRow (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.warehouse (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.time_dim (13) + : +- CometBroadcastExchange (22) + : +- CometProject (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.ship_mode (19) + +- * HashAggregate (46) + +- Exchange (45) + +- * ColumnarToRow (44) + +- CometHashAggregate (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- ReusedExchange (40) (1) Scan parquet spark_catalog.default.web_sales @@ -65,49 +64,53 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] Condition : isnotnull(w_warehouse_sk#9) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -(7) BroadcastExchange -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] +Right output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Arguments: [ws_warehouse_sk#3], [w_warehouse_sk#9], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#3] -Right keys [1]: [w_warehouse_sk#9] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 5] -Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +(7) CometProject Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -(10) ReusedExchange [Reuses operator id: 56] +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#16, d_year#17, d_moy#18] +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 + +(9) CometFilter +Input [3]: [d_date_sk#16, d_year#17, d_moy#18] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#7] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(10) CometBroadcastExchange +Input [3]: [d_date_sk#16, d_year#17, d_moy#18] +Arguments: [d_date_sk#16, d_year#17, d_moy#18] -(12) Project [codegen id : 5] -Output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +(11) CometBroadcastHashJoin +Left output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Right output [3]: [d_date_sk#16, d_year#17, d_moy#18] +Arguments: [ws_sold_date_sk#7], [d_date_sk#16], Inner, BuildRight + +(12) CometProject Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_date_sk#16, d_year#17, d_moy#18] +Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] (13) Scan parquet spark_catalog.default.time_dim Output [2]: [t_time_sk#19, t_time#20] @@ -124,209 +127,196 @@ Condition : (((isnotnull(t_time#20) AND (t_time#20 >= 30838)) AND (t_time#20 <= Input [2]: [t_time_sk#19, t_time#20] Arguments: [t_time_sk#19], [t_time_sk#19] -(16) ColumnarToRow [codegen id : 3] +(16) CometBroadcastExchange Input [1]: [t_time_sk#19] +Arguments: [t_time_sk#19] -(17) BroadcastExchange -Input [1]: [t_time_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +Right output [1]: [t_time_sk#19] +Arguments: [ws_sold_time_sk#1], [t_time_sk#19], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#19] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 5] -Output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +(18) CometProject Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18, t_time_sk#19] +Arguments: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18], [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] -(20) Scan parquet spark_catalog.default.ship_mode +(19) Scan parquet spark_catalog.default.ship_mode Output [2]: [sm_ship_mode_sk#21, sm_carrier#22] Batched: true Location [not included in comparison]/{warehouse_dir}/ship_mode] PushedFilters: [In(sm_carrier, [BARIAN ,DHL ]), IsNotNull(sm_ship_mode_sk)] ReadSchema: struct -(21) CometFilter +(20) CometFilter Input [2]: [sm_ship_mode_sk#21, sm_carrier#22] Condition : (sm_carrier#22 IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#21)) -(22) CometProject +(21) CometProject Input [2]: [sm_ship_mode_sk#21, sm_carrier#22] Arguments: [sm_ship_mode_sk#21], [sm_ship_mode_sk#21] -(23) ColumnarToRow [codegen id : 4] -Input [1]: [sm_ship_mode_sk#21] - -(24) BroadcastExchange +(22) CometBroadcastExchange Input [1]: [sm_ship_mode_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [sm_ship_mode_sk#21] -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_mode_sk#2] -Right keys [1]: [sm_ship_mode_sk#21] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +Right output [1]: [sm_ship_mode_sk#21] +Arguments: [ws_ship_mode_sk#2], [sm_ship_mode_sk#21], Inner, BuildRight -(26) Project [codegen id : 5] -Output [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +(24) CometProject Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18, sm_ship_mode_sk#21] +Arguments: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18], [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] -(27) HashAggregate [codegen id : 5] +(25) CometHashAggregate Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17] Functions [24]: [partial_sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#23, isEmpty#24, sum#25, isEmpty#26, sum#27, isEmpty#28, sum#29, isEmpty#30, sum#31, isEmpty#32, sum#33, isEmpty#34, sum#35, isEmpty#36, sum#37, isEmpty#38, sum#39, isEmpty#40, sum#41, isEmpty#42, sum#43, isEmpty#44, sum#45, isEmpty#46, sum#47, isEmpty#48, sum#49, isEmpty#50, sum#51, isEmpty#52, sum#53, isEmpty#54, sum#55, isEmpty#56, sum#57, isEmpty#58, sum#59, isEmpty#60, sum#61, isEmpty#62, sum#63, isEmpty#64, sum#65, isEmpty#66, sum#67, isEmpty#68, sum#69, isEmpty#70] -Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] -(28) Exchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(26) ColumnarToRow [codegen id : 1] +Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#23, isEmpty#24, sum#25, isEmpty#26, sum#27, isEmpty#28, sum#29, isEmpty#30, sum#31, isEmpty#32, sum#33, isEmpty#34, sum#35, isEmpty#36, sum#37, isEmpty#38, sum#39, isEmpty#40, sum#41, isEmpty#42, sum#43, isEmpty#44, sum#45, isEmpty#46, sum#47, isEmpty#48, sum#49, isEmpty#50, sum#51, isEmpty#52, sum#53, isEmpty#54, sum#55, isEmpty#56, sum#57, isEmpty#58, sum#59, isEmpty#60, sum#61, isEmpty#62, sum#63, isEmpty#64, sum#65, isEmpty#66, sum#67, isEmpty#68, sum#69, isEmpty#70] + +(27) Exchange +Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#23, isEmpty#24, sum#25, isEmpty#26, sum#27, isEmpty#28, sum#29, isEmpty#30, sum#31, isEmpty#32, sum#33, isEmpty#34, sum#35, isEmpty#36, sum#37, isEmpty#38, sum#39, isEmpty#40, sum#41, isEmpty#42, sum#43, isEmpty#44, sum#45, isEmpty#46, sum#47, isEmpty#48, sum#49, isEmpty#50, sum#51, isEmpty#52, sum#53, isEmpty#54, sum#55, isEmpty#56, sum#57, isEmpty#58, sum#59, isEmpty#60, sum#61, isEmpty#62, sum#63, isEmpty#64, sum#65, isEmpty#66, sum#67, isEmpty#68, sum#69, isEmpty#70] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 6] -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] +(28) HashAggregate [codegen id : 2] +Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#23, isEmpty#24, sum#25, isEmpty#26, sum#27, isEmpty#28, sum#29, isEmpty#30, sum#31, isEmpty#32, sum#33, isEmpty#34, sum#35, isEmpty#36, sum#37, isEmpty#38, sum#39, isEmpty#40, sum#41, isEmpty#42, sum#43, isEmpty#44, sum#45, isEmpty#46, sum#47, isEmpty#48, sum#49, isEmpty#50, sum#51, isEmpty#52, sum#53, isEmpty#54, sum#55, isEmpty#56, sum#57, isEmpty#58, sum#59, isEmpty#60, sum#61, isEmpty#62, sum#63, isEmpty#64, sum#65, isEmpty#66, sum#67, isEmpty#68, sum#69, isEmpty#70] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17] Functions [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142] -Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#143, d_year#17 AS year#144, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119 AS jan_sales#145, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120 AS feb_sales#146, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121 AS mar_sales#147, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122 AS apr_sales#148, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123 AS may_sales#149, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124 AS jun_sales#150, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125 AS jul_sales#151, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126 AS aug_sales#152, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127 AS sep_sales#153, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128 AS oct_sales#154, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129 AS nov_sales#155, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130 AS dec_sales#156, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131 AS jan_net#157, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132 AS feb_net#158, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133 AS mar_net#159, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134 AS apr_net#160, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135 AS may_net#161, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136 AS jun_net#162, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137 AS jul_net#163, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138 AS aug_net#164, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139 AS sep_net#165, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140 AS oct_net#166, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141 AS nov_net#167, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142 AS dec_net#168] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#71, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#72, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#73, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#74, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#75, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#76, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#77, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#78, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#79, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#80, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#81, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#82, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#83, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#84, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#85, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#86, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#87, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#88, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#89, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#90, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#91, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#92, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#93, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#94] +Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#95, d_year#17 AS year#96, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#71 AS jan_sales#97, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#72 AS feb_sales#98, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#73 AS mar_sales#99, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#74 AS apr_sales#100, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#75 AS may_sales#101, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#76 AS jun_sales#102, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#77 AS jul_sales#103, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#78 AS aug_sales#104, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#79 AS sep_sales#105, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#80 AS oct_sales#106, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#81 AS nov_sales#107, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#82 AS dec_sales#108, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#83 AS jan_net#109, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#84 AS feb_net#110, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#85 AS mar_net#111, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#86 AS apr_net#112, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#87 AS may_net#113, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#88 AS jun_net#114, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#89 AS jul_net#115, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#90 AS aug_net#116, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#91 AS sep_net#117, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#92 AS oct_net#118, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#93 AS nov_net#119, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#94 AS dec_net#120] -(30) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] +(29) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_warehouse_sk#123, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#175), dynamicpruningexpression(cs_sold_date_sk#175 IN dynamicpruning#176)] +PartitionFilters: [isnotnull(cs_sold_date_sk#127), dynamicpruningexpression(cs_sold_date_sk#127 IN dynamicpruning#128)] PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(31) CometFilter -Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] -Condition : ((isnotnull(cs_warehouse_sk#171) AND isnotnull(cs_sold_time_sk#169)) AND isnotnull(cs_ship_mode_sk#170)) - -(32) ColumnarToRow [codegen id : 11] -Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] - -(33) ReusedExchange [Reuses operator id: 7] -Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_warehouse_sk#171] -Right keys [1]: [w_warehouse_sk#177] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 11] -Output [12]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] - -(36) ReusedExchange [Reuses operator id: 56] -Output [3]: [d_date_sk#184, d_year#185, d_moy#186] - -(37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#175] -Right keys [1]: [d_date_sk#184] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [15]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] - -(39) ReusedExchange [Reuses operator id: 17] -Output [1]: [t_time_sk#187] - -(40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_time_sk#169] -Right keys [1]: [t_time_sk#187] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] - -(42) ReusedExchange [Reuses operator id: 24] -Output [1]: [sm_ship_mode_sk#188] - -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_ship_mode_sk#170] -Right keys [1]: [sm_ship_mode_sk#188] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 11] -Output [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [13]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] - -(45) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] -Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236] -Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] - -(46) Exchange -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(47) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] -Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308] -Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, DHL,BARIAN AS ship_carriers#309, d_year#185 AS year#310, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285 AS jan_sales#311, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286 AS feb_sales#312, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287 AS mar_sales#313, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288 AS apr_sales#314, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289 AS may_sales#315, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290 AS jun_sales#316, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291 AS jul_sales#317, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292 AS aug_sales#318, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293 AS sep_sales#319, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294 AS oct_sales#320, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295 AS nov_sales#321, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296 AS dec_sales#322, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297 AS jan_net#323, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298 AS feb_net#324, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299 AS mar_net#325, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300 AS apr_net#326, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301 AS may_net#327, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302 AS jun_net#328, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303 AS jul_net#329, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304 AS aug_net#330, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305 AS sep_net#331, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306 AS oct_net#332, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307 AS nov_net#333, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308 AS dec_net#334] - -(48) Union - -(49) HashAggregate [codegen id : 13] -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#145, feb_sales#146, mar_sales#147, apr_sales#148, may_sales#149, jun_sales#150, jul_sales#151, aug_sales#152, sep_sales#153, oct_sales#154, nov_sales#155, dec_sales#156, jan_net#157, feb_net#158, mar_net#159, apr_net#160, may_net#161, jun_net#162, jul_net#163, aug_net#164, sep_net#165, oct_net#166, nov_net#167, dec_net#168] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] -Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial_sum(mar_sales#147), partial_sum(apr_sales#148), partial_sum(may_sales#149), partial_sum(jun_sales#150), partial_sum(jul_sales#151), partial_sum(aug_sales#152), partial_sum(sep_sales#153), partial_sum(oct_sales#154), partial_sum(nov_sales#155), partial_sum(dec_sales#156), partial_sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#157), partial_sum(feb_net#158), partial_sum(mar_net#159), partial_sum(apr_net#160), partial_sum(may_net#161), partial_sum(jun_net#162), partial_sum(jul_net#163), partial_sum(aug_net#164), partial_sum(sep_net#165), partial_sum(oct_net#166), partial_sum(nov_net#167), partial_sum(dec_net#168)] -Aggregate Attributes [72]: [sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384, sum#385, isEmpty#386, sum#387, isEmpty#388, sum#389, isEmpty#390, sum#391, isEmpty#392, sum#393, isEmpty#394, sum#395, isEmpty#396, sum#397, isEmpty#398, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406] -Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] - -(50) Exchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(51) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] -Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#157), sum(feb_net#158), sum(mar_net#159), sum(apr_net#160), sum(may_net#161), sum(jun_net#162), sum(jul_net#163), sum(aug_net#164), sum(sep_net#165), sum(oct_net#166), sum(nov_net#167), sum(dec_net#168)] -Aggregate Attributes [36]: [sum(jan_sales#145)#479, sum(feb_sales#146)#480, sum(mar_sales#147)#481, sum(apr_sales#148)#482, sum(may_sales#149)#483, sum(jun_sales#150)#484, sum(jul_sales#151)#485, sum(aug_sales#152)#486, sum(sep_sales#153)#487, sum(oct_sales#154)#488, sum(nov_sales#155)#489, sum(dec_sales#156)#490, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502, sum(jan_net#157)#503, sum(feb_net#158)#504, sum(mar_net#159)#505, sum(apr_net#160)#506, sum(may_net#161)#507, sum(jun_net#162)#508, sum(jul_net#163)#509, sum(aug_net#164)#510, sum(sep_net#165)#511, sum(oct_net#166)#512, sum(nov_net#167)#513, sum(dec_net#168)#514] -Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum(jan_sales#145)#479 AS jan_sales#515, sum(feb_sales#146)#480 AS feb_sales#516, sum(mar_sales#147)#481 AS mar_sales#517, sum(apr_sales#148)#482 AS apr_sales#518, sum(may_sales#149)#483 AS may_sales#519, sum(jun_sales#150)#484 AS jun_sales#520, sum(jul_sales#151)#485 AS jul_sales#521, sum(aug_sales#152)#486 AS aug_sales#522, sum(sep_sales#153)#487 AS sep_sales#523, sum(oct_sales#154)#488 AS oct_sales#524, sum(nov_sales#155)#489 AS nov_sales#525, sum(dec_sales#156)#490 AS dec_sales#526, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491 AS jan_sales_per_sq_foot#527, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492 AS feb_sales_per_sq_foot#528, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493 AS mar_sales_per_sq_foot#529, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494 AS apr_sales_per_sq_foot#530, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495 AS may_sales_per_sq_foot#531, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496 AS jun_sales_per_sq_foot#532, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497 AS jul_sales_per_sq_foot#533, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498 AS aug_sales_per_sq_foot#534, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499 AS sep_sales_per_sq_foot#535, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500 AS oct_sales_per_sq_foot#536, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501 AS nov_sales_per_sq_foot#537, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502 AS dec_sales_per_sq_foot#538, sum(jan_net#157)#503 AS jan_net#539, sum(feb_net#158)#504 AS feb_net#540, sum(mar_net#159)#505 AS mar_net#541, sum(apr_net#160)#506 AS apr_net#542, sum(may_net#161)#507 AS may_net#543, sum(jun_net#162)#508 AS jun_net#544, sum(jul_net#163)#509 AS jul_net#545, sum(aug_net#164)#510 AS aug_net#546, sum(sep_net#165)#511 AS sep_net#547, sum(oct_net#166)#512 AS oct_net#548, sum(nov_net#167)#513 AS nov_net#549, sum(dec_net#168)#514 AS dec_net#550] - -(52) TakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] -Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] +(30) CometFilter +Input [7]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_warehouse_sk#123, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127] +Condition : ((isnotnull(cs_warehouse_sk#123) AND isnotnull(cs_sold_time_sk#121)) AND isnotnull(cs_ship_mode_sk#122)) + +(31) ReusedExchange [Reuses operator id: 5] +Output [7]: [w_warehouse_sk#129, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] + +(32) CometBroadcastHashJoin +Left output [7]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_warehouse_sk#123, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127] +Right output [7]: [w_warehouse_sk#129, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] +Arguments: [cs_warehouse_sk#123], [w_warehouse_sk#129], Inner, BuildRight + +(33) CometProject +Input [14]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_warehouse_sk#123, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127, w_warehouse_sk#129, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] +Arguments: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135], [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] + +(34) ReusedExchange [Reuses operator id: 10] +Output [3]: [d_date_sk#136, d_year#137, d_moy#138] + +(35) CometBroadcastHashJoin +Left output [12]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135] +Right output [3]: [d_date_sk#136, d_year#137, d_moy#138] +Arguments: [cs_sold_date_sk#127], [d_date_sk#136], Inner, BuildRight + +(36) CometProject +Input [15]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, cs_sold_date_sk#127, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_date_sk#136, d_year#137, d_moy#138] +Arguments: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138], [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] + +(37) ReusedExchange [Reuses operator id: 16] +Output [1]: [t_time_sk#139] + +(38) CometBroadcastHashJoin +Left output [13]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] +Right output [1]: [t_time_sk#139] +Arguments: [cs_sold_time_sk#121], [t_time_sk#139], Inner, BuildRight + +(39) CometProject +Input [14]: [cs_sold_time_sk#121, cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138, t_time_sk#139] +Arguments: [cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138], [cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] + +(40) ReusedExchange [Reuses operator id: 22] +Output [1]: [sm_ship_mode_sk#140] + +(41) CometBroadcastHashJoin +Left output [12]: [cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] +Right output [1]: [sm_ship_mode_sk#140] +Arguments: [cs_ship_mode_sk#122], [sm_ship_mode_sk#140], Inner, BuildRight + +(42) CometProject +Input [13]: [cs_ship_mode_sk#122, cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138, sm_ship_mode_sk#140] +Arguments: [cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138], [cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] + +(43) CometHashAggregate +Input [11]: [cs_quantity#124, cs_sales_price#125, cs_net_paid_inc_tax#126, w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, d_moy#138] +Keys [7]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137] +Functions [24]: [partial_sum(CASE WHEN (d_moy#138 = 1) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 2) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 3) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 4) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 5) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 6) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 7) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 8) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 9) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 10) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 11) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 12) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 1) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 2) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 3) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 4) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 5) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 6) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 7) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 8) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 9) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 10) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 11) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#138 = 12) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)] + +(44) ColumnarToRow [codegen id : 3] +Input [55]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] + +(45) Exchange +Input [55]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] +Arguments: hashpartitioning(w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(46) HashAggregate [codegen id : 4] +Input [55]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] +Keys [7]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, d_year#137] +Functions [24]: [sum(CASE WHEN (d_moy#138 = 1) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 2) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 3) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 4) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 5) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 6) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 7) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 8) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 9) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 10) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 11) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 12) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 1) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 2) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 3) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 4) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 5) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 6) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 7) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 8) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 9) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 10) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 11) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#138 = 12) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#138 = 1) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#189, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#190, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#191, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#192, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#193, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#194, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#195, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#196, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#197, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#198, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#199, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#200, sum(CASE WHEN (d_moy#138 = 1) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#201, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#202, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#203, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#204, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#205, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#206, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#207, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#208, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#209, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#210, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#211, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#212] +Results [32]: [w_warehouse_name#130, w_warehouse_sq_ft#131, w_city#132, w_county#133, w_state#134, w_country#135, DHL,BARIAN AS ship_carriers#213, d_year#137 AS year#214, sum(CASE WHEN (d_moy#138 = 1) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#189 AS jan_sales#215, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#190 AS feb_sales#216, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#191 AS mar_sales#217, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#192 AS apr_sales#218, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#193 AS may_sales#219, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#194 AS jun_sales#220, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#195 AS jul_sales#221, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#196 AS aug_sales#222, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#197 AS sep_sales#223, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#198 AS oct_sales#224, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#199 AS nov_sales#225, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_sales_price#125 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#200 AS dec_sales#226, sum(CASE WHEN (d_moy#138 = 1) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#201 AS jan_net#227, sum(CASE WHEN (d_moy#138 = 2) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#202 AS feb_net#228, sum(CASE WHEN (d_moy#138 = 3) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#203 AS mar_net#229, sum(CASE WHEN (d_moy#138 = 4) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#204 AS apr_net#230, sum(CASE WHEN (d_moy#138 = 5) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#205 AS may_net#231, sum(CASE WHEN (d_moy#138 = 6) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#206 AS jun_net#232, sum(CASE WHEN (d_moy#138 = 7) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#207 AS jul_net#233, sum(CASE WHEN (d_moy#138 = 8) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#208 AS aug_net#234, sum(CASE WHEN (d_moy#138 = 9) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#209 AS sep_net#235, sum(CASE WHEN (d_moy#138 = 10) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#210 AS oct_net#236, sum(CASE WHEN (d_moy#138 = 11) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#211 AS nov_net#237, sum(CASE WHEN (d_moy#138 = 12) THEN (cs_net_paid_inc_tax#126 * cast(cs_quantity#124 as decimal(10,0))) ELSE 0.00 END)#212 AS dec_net#238] + +(47) Union + +(48) HashAggregate [codegen id : 5] +Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, jan_sales#97, feb_sales#98, mar_sales#99, apr_sales#100, may_sales#101, jun_sales#102, jul_sales#103, aug_sales#104, sep_sales#105, oct_sales#106, nov_sales#107, dec_sales#108, jan_net#109, feb_net#110, mar_net#111, apr_net#112, may_net#113, jun_net#114, jul_net#115, aug_net#116, sep_net#117, oct_net#118, nov_net#119, dec_net#120] +Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96] +Functions [36]: [partial_sum(jan_sales#97), partial_sum(feb_sales#98), partial_sum(mar_sales#99), partial_sum(apr_sales#100), partial_sum(may_sales#101), partial_sum(jun_sales#102), partial_sum(jul_sales#103), partial_sum(aug_sales#104), partial_sum(sep_sales#105), partial_sum(oct_sales#106), partial_sum(nov_sales#107), partial_sum(dec_sales#108), partial_sum((jan_sales#97 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#98 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#99 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#100 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#101 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#102 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#103 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#104 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#105 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#106 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#107 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#108 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#109), partial_sum(feb_net#110), partial_sum(mar_net#111), partial_sum(apr_net#112), partial_sum(may_net#113), partial_sum(jun_net#114), partial_sum(jul_net#115), partial_sum(aug_net#116), partial_sum(sep_net#117), partial_sum(oct_net#118), partial_sum(nov_net#119), partial_sum(dec_net#120)] +Aggregate Attributes [72]: [sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288, sum#289, isEmpty#290, sum#291, isEmpty#292, sum#293, isEmpty#294, sum#295, isEmpty#296, sum#297, isEmpty#298, sum#299, isEmpty#300, sum#301, isEmpty#302, sum#303, isEmpty#304, sum#305, isEmpty#306, sum#307, isEmpty#308, sum#309, isEmpty#310] +Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum#311, isEmpty#312, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382] + +(49) Exchange +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum#311, isEmpty#312, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(50) HashAggregate [codegen id : 6] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum#311, isEmpty#312, sum#313, isEmpty#314, sum#315, isEmpty#316, sum#317, isEmpty#318, sum#319, isEmpty#320, sum#321, isEmpty#322, sum#323, isEmpty#324, sum#325, isEmpty#326, sum#327, isEmpty#328, sum#329, isEmpty#330, sum#331, isEmpty#332, sum#333, isEmpty#334, sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382] +Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96] +Functions [36]: [sum(jan_sales#97), sum(feb_sales#98), sum(mar_sales#99), sum(apr_sales#100), sum(may_sales#101), sum(jun_sales#102), sum(jul_sales#103), sum(aug_sales#104), sum(sep_sales#105), sum(oct_sales#106), sum(nov_sales#107), sum(dec_sales#108), sum((jan_sales#97 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#98 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#99 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#100 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#101 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#102 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#103 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#104 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#105 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#106 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#107 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#108 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#109), sum(feb_net#110), sum(mar_net#111), sum(apr_net#112), sum(may_net#113), sum(jun_net#114), sum(jul_net#115), sum(aug_net#116), sum(sep_net#117), sum(oct_net#118), sum(nov_net#119), sum(dec_net#120)] +Aggregate Attributes [36]: [sum(jan_sales#97)#383, sum(feb_sales#98)#384, sum(mar_sales#99)#385, sum(apr_sales#100)#386, sum(may_sales#101)#387, sum(jun_sales#102)#388, sum(jul_sales#103)#389, sum(aug_sales#104)#390, sum(sep_sales#105)#391, sum(oct_sales#106)#392, sum(nov_sales#107)#393, sum(dec_sales#108)#394, sum((jan_sales#97 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#395, sum((feb_sales#98 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#396, sum((mar_sales#99 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#397, sum((apr_sales#100 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#398, sum((may_sales#101 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#399, sum((jun_sales#102 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#400, sum((jul_sales#103 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#401, sum((aug_sales#104 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#402, sum((sep_sales#105 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#403, sum((oct_sales#106 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#404, sum((nov_sales#107 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#405, sum((dec_sales#108 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#406, sum(jan_net#109)#407, sum(feb_net#110)#408, sum(mar_net#111)#409, sum(apr_net#112)#410, sum(may_net#113)#411, sum(jun_net#114)#412, sum(jul_net#115)#413, sum(aug_net#116)#414, sum(sep_net#117)#415, sum(oct_net#118)#416, sum(nov_net#119)#417, sum(dec_net#120)#418] +Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, sum(jan_sales#97)#383 AS jan_sales#419, sum(feb_sales#98)#384 AS feb_sales#420, sum(mar_sales#99)#385 AS mar_sales#421, sum(apr_sales#100)#386 AS apr_sales#422, sum(may_sales#101)#387 AS may_sales#423, sum(jun_sales#102)#388 AS jun_sales#424, sum(jul_sales#103)#389 AS jul_sales#425, sum(aug_sales#104)#390 AS aug_sales#426, sum(sep_sales#105)#391 AS sep_sales#427, sum(oct_sales#106)#392 AS oct_sales#428, sum(nov_sales#107)#393 AS nov_sales#429, sum(dec_sales#108)#394 AS dec_sales#430, sum((jan_sales#97 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#395 AS jan_sales_per_sq_foot#431, sum((feb_sales#98 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#396 AS feb_sales_per_sq_foot#432, sum((mar_sales#99 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#397 AS mar_sales_per_sq_foot#433, sum((apr_sales#100 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#398 AS apr_sales_per_sq_foot#434, sum((may_sales#101 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#399 AS may_sales_per_sq_foot#435, sum((jun_sales#102 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#400 AS jun_sales_per_sq_foot#436, sum((jul_sales#103 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#401 AS jul_sales_per_sq_foot#437, sum((aug_sales#104 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#402 AS aug_sales_per_sq_foot#438, sum((sep_sales#105 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#403 AS sep_sales_per_sq_foot#439, sum((oct_sales#106 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#404 AS oct_sales_per_sq_foot#440, sum((nov_sales#107 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#405 AS nov_sales_per_sq_foot#441, sum((dec_sales#108 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#406 AS dec_sales_per_sq_foot#442, sum(jan_net#109)#407 AS jan_net#443, sum(feb_net#110)#408 AS feb_net#444, sum(mar_net#111)#409 AS mar_net#445, sum(apr_net#112)#410 AS apr_net#446, sum(may_net#113)#411 AS may_net#447, sum(jun_net#114)#412 AS jun_net#448, sum(jul_net#115)#413 AS jul_net#449, sum(aug_net#116)#414 AS aug_net#450, sum(sep_net#117)#415 AS sep_net#451, sum(oct_net#118)#416 AS oct_net#452, sum(nov_net#119)#417 AS nov_net#453, sum(dec_net#120)#418 AS dec_net#454] + +(51) TakeOrderedAndProject +Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, jan_sales#419, feb_sales#420, mar_sales#421, apr_sales#422, may_sales#423, jun_sales#424, jul_sales#425, aug_sales#426, sep_sales#427, oct_sales#428, nov_sales#429, dec_sales#430, jan_sales_per_sq_foot#431, feb_sales_per_sq_foot#432, mar_sales_per_sq_foot#433, apr_sales_per_sq_foot#434, may_sales_per_sq_foot#435, jun_sales_per_sq_foot#436, jul_sales_per_sq_foot#437, aug_sales_per_sq_foot#438, sep_sales_per_sq_foot#439, oct_sales_per_sq_foot#440, nov_sales_per_sq_foot#441, dec_sales_per_sq_foot#442, jan_net#443, feb_net#444, mar_net#445, apr_net#446, may_net#447, jun_net#448, jul_net#449, aug_net#450, sep_net#451, oct_net#452, nov_net#453, dec_net#454] +Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#95, year#96, jan_sales#419, feb_sales#420, mar_sales#421, apr_sales#422, may_sales#423, jun_sales#424, jul_sales#425, aug_sales#426, sep_sales#427, oct_sales#428, nov_sales#429, dec_sales#430, jan_sales_per_sq_foot#431, feb_sales_per_sq_foot#432, mar_sales_per_sq_foot#433, apr_sales_per_sq_foot#434, may_sales_per_sq_foot#435, jun_sales_per_sq_foot#436, jul_sales_per_sq_foot#437, aug_sales_per_sq_foot#438, sep_sales_per_sq_foot#439, oct_sales_per_sq_foot#440, nov_sales_per_sq_foot#441, dec_sales_per_sq_foot#442, jan_net#443, feb_net#444, mar_net#445, apr_net#446, may_net#447, jun_net#448, jul_net#449, aug_net#450, sep_net#451, oct_net#452, nov_net#453, dec_net#454] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (56) -+- * ColumnarToRow (55) - +- CometFilter (54) - +- CometScan parquet spark_catalog.default.date_dim (53) +BroadcastExchange (55) ++- * ColumnarToRow (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) -(53) Scan parquet spark_catalog.default.date_dim +(52) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#16, d_year#17, d_moy#18] 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 -(54) CometFilter +(53) CometFilter Input [3]: [d_date_sk#16, d_year#17, d_moy#18] Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) -(55) ColumnarToRow [codegen id : 1] +(54) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -(56) BroadcastExchange +(55) BroadcastExchange Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#175 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#127 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt index 8ed74582f0..d746739b33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt @@ -1,28 +1,28 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) + WholeStageCodegen (6) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) + WholeStageCodegen (5) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometProject [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + CometProject [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometBroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -32,55 +32,40 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_time,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [sm_ship_mode_sk] - CometFilter [sm_carrier,sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) + CometBroadcastExchange #4 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometBroadcastExchange #5 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometProject [t_time_sk] + CometFilter [t_time,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + CometBroadcastExchange #7 + CometProject [sm_ship_mode_sk] + CometFilter [sm_carrier,sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometProject [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometProject [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [cs_sold_time_sk,t_time_sk] + CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - ReusedExchange [t_time_sk] #5 - InputAdapter - ReusedExchange [sm_ship_mode_sk] #6 + ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + ReusedExchange [d_date_sk,d_year,d_moy] #5 + ReusedExchange [t_time_sk] #6 + ReusedExchange [sm_ship_mode_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index 53e9b00bf8..9301e7b8f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -1,31 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (27) -+- * Filter (26) - +- Window (25) - +- * Sort (24) - +- Exchange (23) - +- * HashAggregate (22) - +- Exchange (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.item (13) +TakeOrderedAndProject (28) ++- * Filter (27) + +- Window (26) + +- * Sort (25) + +- Exchange (24) + +- * HashAggregate (23) + +- Exchange (22) + +- * ColumnarToRow (21) + +- CometHashAggregate (20) + +- CometExpand (19) + +- CometProject (18) + +- CometBroadcastHashJoin (17) + :- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.store (9) + +- CometBroadcastExchange (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.item (14) (1) Scan parquet spark_catalog.default.store_sales @@ -40,150 +41,155 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 32] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(4) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(6) CometBroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] +Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight + +(8) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] + +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#12, s_store_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) - -(9) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#12] +(10) CometFilter +Input [2]: [s_store_sk#12, s_store_id#13] +Condition : isnotnull(s_store_sk#12) -(10) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [2]: [s_store_sk#12, s_store_id#13] +Arguments: [s_store_sk#12, s_store_id#13] -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] +Right output [2]: [s_store_sk#12, s_store_id#13] +Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight -(12) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#12] +(13) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#13] +Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13] -(13) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +(14) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(14) CometFilter -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Condition : isnotnull(i_item_sk#13) +(15) CometFilter +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Condition : isnotnull(i_item_sk#14) -(15) ColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +(16) CometBroadcastExchange +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -(16) BroadcastExchange -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13] +Right output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [ss_item_sk#1], [i_item_sk#14], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(18) CometProject +Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] -(18) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +(19) CometExpand +Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] +Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, 0], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#17, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] -(19) Expand [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] -Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 0], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#16, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] - -(20) HashAggregate [codegen id : 4] -Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] -Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] +(20) CometHashAggregate +Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] +Keys [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#27, isEmpty#28] -Results [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] -(21) Exchange -Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] -Arguments: hashpartitioning(i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) ColumnarToRow [codegen id : 1] +Input [11]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27, sum#28, isEmpty#29] + +(22) Exchange +Input [11]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27, sum#28, isEmpty#29] +Arguments: hashpartitioning(i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] -Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] -Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] +(23) HashAggregate [codegen id : 2] +Input [11]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27, sum#28, isEmpty#29] +Keys [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31] -Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31 AS sumsales#32] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#30] +Results [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#30 AS sumsales#31] -(23) Exchange -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(24) Exchange +Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#31] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(24) Sort [codegen id : 6] -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0 +(25) Sort [codegen id : 3] +Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#31] +Arguments: [i_category#19 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 -(25) Window -Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] -Arguments: [rank(sumsales#32) windowspecdefinition(i_category#18, sumsales#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#33], [i_category#18], [sumsales#32 DESC NULLS LAST] +(26) Window +Input [9]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#31] +Arguments: [rank(sumsales#31) windowspecdefinition(i_category#19, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [i_category#19], [sumsales#31 DESC NULLS LAST] -(26) Filter [codegen id : 7] -Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] -Condition : (rk#33 <= 100) +(27) Filter [codegen id : 4] +Input [10]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#31, rk#32] +Condition : (rk#32 <= 100) -(27) TakeOrderedAndProject -Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] -Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_qoy#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, sumsales#32 ASC NULLS FIRST, rk#33 ASC NULLS FIRST], [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] +(28) TakeOrderedAndProject +Input [10]: [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#31, rk#32] +Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, i_product_name#22 ASC NULLS FIRST, d_year#23 ASC NULLS FIRST, d_qoy#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, s_store_id#26 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#32 ASC NULLS FIRST], [i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, sumsales#31, rk#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (32) -+- * ColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan parquet spark_catalog.default.date_dim (28) +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) -(28) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] +(29) Scan parquet spark_catalog.default.date_dim +Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(29) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#7)) +(30) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) -(30) CometProject -Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(31) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(31) ColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(32) ColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(32) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) BroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index 119ae71840..b529cb5ff3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -1,27 +1,27 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [rk] InputAdapter Window [sumsales,i_category] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,sumsales] InputAdapter Exchange [i_category] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] + CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] + CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -32,19 +32,13 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange #5 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index 734b6c11c6..53443aedc7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -1,43 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) +TakeOrderedAndProject (42) ++- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) :- * Project (35) : +- * BroadcastHashJoin Inner BuildRight (34) : :- * HashAggregate (29) : : +- Exchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * ColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * ColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * ColumnarToRow (23) + : : +- * ColumnarToRow (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometProject (20) + : : : +- CometBroadcastHashJoin (19) + : : : :- CometProject (14) + : : : : +- CometBroadcastHashJoin (13) + : : : : :- CometProject (8) + : : : : : +- CometBroadcastHashJoin (7) + : : : : : :- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- CometBroadcastExchange (6) + : : : : : +- CometProject (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : : +- CometBroadcastExchange (12) + : : : : +- CometProject (11) + : : : : +- CometFilter (10) + : : : : +- CometScan parquet spark_catalog.default.store (9) + : : : +- CometBroadcastExchange (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.household_demographics (15) + : : +- CometBroadcastExchange (23) : : +- CometFilter (22) : : +- CometScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (33) : +- * ColumnarToRow (32) : +- CometFilter (31) : +- CometScan parquet spark_catalog.default.customer (30) - +- ReusedExchange (36) + +- BroadcastExchange (39) + +- * ColumnarToRow (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.customer_address (36) (1) Scan parquet spark_catalog.default.store_sales @@ -52,207 +55,223 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#11] +(4) CometFilter +Input [3]: [d_date_sk#11, d_year#12, d_dom#13] +Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#11, d_year#12, d_dom#13] +Arguments: [d_date_sk#11], [d_date_sk#11] -(6) Project [codegen id : 5] -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(7) CometBroadcastHashJoin +Left output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] +Right output [1]: [d_date_sk#11] +Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight + +(8) CometProject Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_city#13] +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#14, s_city#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#12, s_city#13] -Condition : (s_city#13 IN (Midway,Fairview) AND isnotnull(s_store_sk#12)) +(10) CometFilter +Input [2]: [s_store_sk#14, s_city#15] +Condition : (s_city#15 IN (Midway,Fairview) AND isnotnull(s_store_sk#14)) -(9) CometProject -Input [2]: [s_store_sk#12, s_city#13] -Arguments: [s_store_sk#12], [s_store_sk#12] +(11) CometProject +Input [2]: [s_store_sk#14, s_city#15] +Arguments: [s_store_sk#14], [s_store_sk#14] -(10) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#12] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#14] +Arguments: [s_store_sk#14] -(11) BroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(13) CometBroadcastHashJoin +Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] +Right output [1]: [s_store_sk#14] +Arguments: [ss_store_sk#4], [s_store_sk#14], Inner, BuildRight -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#12] +(14) CometProject +Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#14] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -(14) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] +(15) Scan parquet spark_catalog.default.household_demographics +Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : (((hd_dep_count#15 = 4) OR (hd_vehicle_count#16 = 3)) AND isnotnull(hd_demo_sk#14)) - -(16) CometProject -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Arguments: [hd_demo_sk#14], [hd_demo_sk#14] +(16) CometFilter +Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] +Condition : (((hd_dep_count#17 = 4) OR (hd_vehicle_count#18 = 3)) AND isnotnull(hd_demo_sk#16)) -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#14] +(17) CometProject +Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] +Arguments: [hd_demo_sk#16], [hd_demo_sk#16] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#16] +Arguments: [hd_demo_sk#16] -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] +Right output [1]: [hd_demo_sk#16] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight -(20) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#14] +(20) CometProject +Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#16] +Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] (21) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_city#18] +Output [2]: [ca_address_sk#19, ca_city#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] ReadSchema: struct (22) CometFilter -Input [2]: [ca_address_sk#17, ca_city#18] -Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_city#18)) +Input [2]: [ca_address_sk#19, ca_city#20] +Condition : (isnotnull(ca_address_sk#19) AND isnotnull(ca_city#20)) -(23) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#17, ca_city#18] - -(24) BroadcastExchange -Input [2]: [ca_address_sk#17, ca_city#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastExchange +Input [2]: [ca_address_sk#19, ca_city#20] +Arguments: [ca_address_sk#19, ca_city#20] -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None +(24) CometBroadcastHashJoin +Left output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] +Right output [2]: [ca_address_sk#19, ca_city#20] +Arguments: [ss_addr_sk#3], [ca_address_sk#19], Inner, BuildRight -(26) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] -Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#17, ca_city#18] +(25) CometProject +Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#19, ca_city#20] +Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] -(27) HashAggregate [codegen id : 5] -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] +(26) CometHashAggregate +Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum#19, sum#20, sum#21] -Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] + +(27) ColumnarToRow [codegen id : 1] +Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] (28) Exchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 8] -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] +(29) HashAggregate [codegen id : 4] +Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#25, sum(UnscaledValue(ss_ext_list_price#7))#26, sum(UnscaledValue(ss_ext_tax#8))#27] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#18 AS bought_city#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#25,17,2) AS extended_price#29, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#26,17,2) AS list_price#30, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#27,17,2) AS extended_tax#31] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#24, sum(UnscaledValue(ss_ext_list_price#7))#25, sum(UnscaledValue(ss_ext_tax#8))#26] +Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#20 AS bought_city#27, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#24,17,2) AS extended_price#28, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#25,17,2) AS list_price#29, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#26,17,2) AS extended_tax#30] (30) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +Output [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (31) CometFilter -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) +Input [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] +Condition : (isnotnull(c_customer_sk#31) AND isnotnull(c_current_addr_sk#32)) -(32) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +(32) ColumnarToRow [codegen id : 2] +Input [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] (33) BroadcastExchange -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +Input [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(34) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#32] +Right keys [1]: [c_customer_sk#31] Join type: Inner Join condition: None -(35) Project [codegen id : 8] -Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +(35) Project [codegen id : 4] +Output [8]: [ss_ticket_number#5, bought_city#27, extended_price#28, list_price#29, extended_tax#30, c_current_addr_sk#32, c_first_name#33, c_last_name#34] +Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#27, extended_price#28, list_price#29, extended_tax#30, c_customer_sk#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] -(36) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#36, ca_city#37] +(36) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#35, ca_city#36] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] +ReadSchema: struct + +(37) CometFilter +Input [2]: [ca_address_sk#35, ca_city#36] +Condition : (isnotnull(ca_address_sk#35) AND isnotnull(ca_city#36)) + +(38) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#35, ca_city#36] + +(39) BroadcastExchange +Input [2]: [ca_address_sk#35, ca_city#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#36] +(40) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [c_current_addr_sk#32] +Right keys [1]: [ca_address_sk#35] Join type: Inner -Join condition: NOT (ca_city#37 = bought_city#28) +Join condition: NOT (ca_city#36 = bought_city#27) -(38) Project [codegen id : 8] -Output [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35, ca_address_sk#36, ca_city#37] +(41) Project [codegen id : 4] +Output [8]: [c_last_name#34, c_first_name#33, ca_city#36, bought_city#27, ss_ticket_number#5, extended_price#28, extended_tax#30, list_price#29] +Input [10]: [ss_ticket_number#5, bought_city#27, extended_price#28, list_price#29, extended_tax#30, c_current_addr_sk#32, c_first_name#33, c_last_name#34, ca_address_sk#35, ca_city#36] -(39) TakeOrderedAndProject -Input [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Arguments: 100, [c_last_name#35 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] +(42) TakeOrderedAndProject +Input [8]: [c_last_name#34, c_first_name#33, ca_city#36, bought_city#27, ss_ticket_number#5, extended_price#28, extended_tax#30, list_price#29] +Arguments: 100, [c_last_name#34 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, ca_city#36, bought_city#27, ss_ticket_number#5, extended_price#28, extended_tax#30, list_price#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(40) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#38, d_dom#39] +(43) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#11, d_year#12, d_dom#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [3]: [d_date_sk#11, d_year#38, d_dom#39] -Condition : ((((isnotnull(d_dom#39) AND (d_dom#39 >= 1)) AND (d_dom#39 <= 2)) AND d_year#38 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) +(44) CometFilter +Input [3]: [d_date_sk#11, d_year#12, d_dom#13] +Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) -(42) CometProject -Input [3]: [d_date_sk#11, d_year#38, d_dom#39] +(45) CometProject +Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(43) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(44) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt index f2680bebb0..43f44c9f74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - WholeStageCodegen (8) + WholeStageCodegen (4) Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] @@ -7,18 +7,18 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] InputAdapter Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] + CometBroadcastHashJoin [ss_addr_sk,ca_address_sk] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,37 +29,32 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ CometProject [d_date_sk] CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_city,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange #4 + CometProject [s_store_sk] + CometFilter [s_city,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometBroadcastExchange #5 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #6 + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 + BroadcastExchange #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index d56b01d737..b931de2923 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (25) - : : +- * BroadcastHashJoin LeftAnti BuildRight (24) - : : :- * BroadcastHashJoin LeftAnti BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.customer_address (26) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (27) + : : +- * BroadcastHashJoin LeftAnti BuildRight (26) + : : :- * BroadcastHashJoin LeftAnti BuildRight (19) + : : : :- * ColumnarToRow (12) + : : : : +- CometBroadcastHashJoin (11) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (10) + : : : : +- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometProject (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer_address (28) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) Scan parquet spark_catalog.default.customer @@ -54,228 +56,237 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#7] +(5) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(6) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(7) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] +(9) CometProject Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#4] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ss_customer_sk#4] +Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight + +(12) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(13) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(13) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#11] +(15) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(16) CometProject +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] +(17) ColumnarToRow [codegen id : 1] +Input [1]: [ws_bill_customer_sk#10] -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) BroadcastExchange +Input [1]: [ws_bill_customer_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 9] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#8] +Right keys [1]: [ws_bill_customer_sk#10] Join type: LeftAnti Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#17] -(20) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#15] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +Right output [1]: [d_date_sk#17] +Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(23) CometProject +Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] +Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#12] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15] +(24) ColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#14] -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#12] +Right keys [1]: [cs_ship_customer_sk#14] Join type: LeftAnti Join condition: None -(25) Project [codegen id : 9] +(27) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -(26) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] +(28) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [GA,KY,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct -(27) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : (ca_state#17 IN (KY,GA,NM) AND isnotnull(ca_address_sk#16)) +(29) CometFilter +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) -(28) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] +(30) CometProject +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: [ca_address_sk#18], [ca_address_sk#18] -(29) ColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#16] +(31) ColumnarToRow [codegen id : 3] +Input [1]: [ca_address_sk#18] -(30) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(32) BroadcastExchange +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(31) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#16] +Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 9] +(34) Project [codegen id : 5] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] -(33) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +(35) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(34) CometFilter -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Condition : isnotnull(cd_demo_sk#18) +(36) CometFilter +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Condition : isnotnull(cd_demo_sk#20) -(35) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +(37) ColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -(36) BroadcastExchange -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(38) BroadcastExchange +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] +Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(38) Project [codegen id : 9] -Output [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +(40) Project [codegen id : 5] +Output [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -(39) HashAggregate [codegen id : 9] -Input [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Keys [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +(41) HashAggregate [codegen id : 5] +Input [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] +Keys [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#24] -Results [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] +Aggregate Attributes [1]: [count#26] +Results [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#27] -(40) Exchange -Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] -Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(42) Exchange +Input [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#27] +Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(41) HashAggregate [codegen id : 10] -Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] -Keys [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +(43) HashAggregate [codegen id : 6] +Input [6]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, count#27] +Keys [5]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#26] -Results [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, count(1)#26 AS cnt1#27, cd_purchase_estimate#22, count(1)#26 AS cnt2#28, cd_credit_rating#23, count(1)#26 AS cnt3#29] +Aggregate Attributes [1]: [count(1)#28] +Results [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#28 AS cnt1#29, cd_purchase_estimate#24, count(1)#28 AS cnt2#30, cd_credit_rating#25, count(1)#28 AS cnt3#31] -(42) TakeOrderedAndProject -Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] -Arguments: 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] +(44) TakeOrderedAndProject +Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#29, cd_purchase_estimate#24, cnt2#30, cd_credit_rating#25, cnt3#31] +Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#29, cd_purchase_estimate#24, cnt2#30, cd_credit_rating#25, cnt3#31] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(43) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#30, d_moy#31] +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter -Input [3]: [d_date_sk#7, d_year#30, d_moy#31] -Condition : (((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2001)) AND (d_moy#31 >= 4)) AND (d_moy#31 <= 6)) AND isnotnull(d_date_sk#7)) +(46) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) -(45) CometProject -Input [3]: [d_date_sk#7, d_year#30, d_moy#31] +(47) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index f5b4eccfbe..de2d5eeda6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (10) + WholeStageCodegen (6) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] InputAdapter Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (9) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -12,62 +12,58 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [ca_address_sk] CometFilter [ca_state,ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt index 790a917e38..93faa6420b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt @@ -2,30 +2,30 @@ TakeOrderedAndProject (30) +- * HashAggregate (29) +- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * ColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) + +- * ColumnarToRow (27) + +- CometHashAggregate (26) + +- CometProject (25) + +- CometBroadcastHashJoin (24) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.date_dim (9) + : +- CometBroadcastExchange (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.item (15) + +- CometBroadcastExchange (23) +- CometProject (22) +- CometFilter (21) +- CometScan parquet spark_catalog.default.promotion (20) @@ -43,135 +43,136 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) ColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#10] -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [cd_demo_sk#10] +Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +(8) CometProject Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] + +(9) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +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 + +(10) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) + +(11) CometProject +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14], [d_date_sk#14] -(11) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#14] +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +(14) CometProject Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(14) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] +(15) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#16, i_item_id#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : isnotnull(i_item_sk#15) - -(16) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#15, i_item_id#16] +(16) CometFilter +Input [2]: [i_item_sk#16, i_item_id#17] +Condition : isnotnull(i_item_sk#16) -(17) BroadcastExchange -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [2]: [i_item_sk#16, i_item_id#17] +Arguments: [i_item_sk#16, i_item_id#17] -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [2]: [i_item_sk#16, i_item_id#17] +Arguments: [ss_item_sk#1], [i_item_sk#16], Inner, BuildRight -(19) Project [codegen id : 5] -Output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#15, i_item_id#16] +(19) CometProject +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#16, i_item_id#17] +Arguments: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17], [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] (20) Scan parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] +Output [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [Or(EqualTo(p_channel_email,N),EqualTo(p_channel_event,N)), IsNotNull(p_promo_sk)] ReadSchema: struct (21) CometFilter -Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] -Condition : (((p_channel_email#18 = N) OR (p_channel_event#19 = N)) AND isnotnull(p_promo_sk#17)) +Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] +Condition : (((p_channel_email#19 = N) OR (p_channel_event#20 = N)) AND isnotnull(p_promo_sk#18)) (22) CometProject -Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] -Arguments: [p_promo_sk#17], [p_promo_sk#17] +Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] +Arguments: [p_promo_sk#18], [p_promo_sk#18] -(23) ColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#17] +(23) CometBroadcastExchange +Input [1]: [p_promo_sk#18] +Arguments: [p_promo_sk#18] -(24) BroadcastExchange -Input [1]: [p_promo_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(24) CometBroadcastHashJoin +Left output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] +Right output [1]: [p_promo_sk#18] +Arguments: [ss_promo_sk#3], [p_promo_sk#18], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#17] -Join type: Inner -Join condition: None +(25) CometProject +Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17, p_promo_sk#18] +Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -(26) Project [codegen id : 5] -Output [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16] -Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16, p_promo_sk#17] - -(27) HashAggregate [codegen id : 5] -Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16] -Keys [1]: [i_item_id#16] +(26) CometHashAggregate +Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] +Keys [1]: [i_item_id#17] Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [8]: [sum#20, count#21, sum#22, count#23, sum#24, count#25, sum#26, count#27] -Results [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] + +(27) ColumnarToRow [codegen id : 1] +Input [9]: [i_item_id#17, sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] (28) Exchange -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [9]: [i_item_id#17, sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] +Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(29) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [1]: [i_item_id#16] +(29) HashAggregate [codegen id : 2] +Input [9]: [i_item_id#17, sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] +Keys [1]: [i_item_id#17] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#36, avg(UnscaledValue(ss_list_price#5))#37, avg(UnscaledValue(ss_coupon_amt#7))#38, avg(UnscaledValue(ss_sales_price#6))#39] -Results [5]: [i_item_id#16, avg(ss_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(ss_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(ss_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(ss_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] +Aggregate Attributes [4]: [avg(ss_quantity#4)#29, avg(UnscaledValue(ss_list_price#5))#30, avg(UnscaledValue(ss_coupon_amt#7))#31, avg(UnscaledValue(ss_sales_price#6))#32] +Results [5]: [i_item_id#17, avg(ss_quantity#4)#29 AS agg1#33, cast((avg(UnscaledValue(ss_list_price#5))#30 / 100.0) as decimal(11,6)) AS agg2#34, cast((avg(UnscaledValue(ss_coupon_amt#7))#31 / 100.0) as decimal(11,6)) AS agg3#35, cast((avg(UnscaledValue(ss_sales_price#6))#32 / 100.0) as decimal(11,6)) AS agg4#36] (30) TakeOrderedAndProject -Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] -Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] +Input [5]: [i_item_id#17, agg1#33, agg2#34, agg3#35, agg4#36] +Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#33, agg2#34, agg3#35, agg4#36] ===== Subqueries ===== @@ -184,18 +185,18 @@ BroadcastExchange (35) (31) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] 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 (32) CometFilter -Input [2]: [d_date_sk#14, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) (33) CometProject -Input [2]: [d_date_sk#14, d_year#44] +Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] (34) ColumnarToRow [codegen id : 1] @@ -203,6 +204,6 @@ Input [1]: [d_date_sk#14] (35) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt index 2471de20a3..c583ba8e8d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + CometBroadcastHashJoin [ss_promo_sk,p_promo_sk] + CometProject [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -25,28 +25,18 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_channel_email,p_channel_event,p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometBroadcastExchange #3 + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #5 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #6 + CometProject [p_promo_sk] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 82deb80a4c..6198dca1e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- Window (40) - +- * Sort (39) - +- Exchange (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Expand (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (31) - +- * BroadcastHashJoin LeftSemi BuildRight (30) - :- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (29) - +- * Project (28) - +- * Filter (27) - +- Window (26) - +- * Sort (25) - +- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildRight (20) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.store (13) - +- ReusedExchange (19) +TakeOrderedAndProject (44) ++- * Project (43) + +- Window (42) + +- * Sort (41) + +- Exchange (40) + +- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Expand (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.date_dim (3) + +- BroadcastExchange (33) + +- * BroadcastHashJoin LeftSemi BuildRight (32) + :- * ColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.store (10) + +- BroadcastExchange (31) + +- * Project (30) + +- * Filter (29) + +- Window (28) + +- * Sort (27) + +- * HashAggregate (26) + +- Exchange (25) + +- * ColumnarToRow (24) + +- CometHashAggregate (23) + +- CometProject (22) + +- CometBroadcastHashJoin (21) + :- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store_sales (13) + : +- CometBroadcastExchange (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.store (15) + +- ReusedExchange (20) (1) Scan parquet spark_catalog.default.store_sales @@ -55,224 +57,232 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(7) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] +(9) ColumnarToRow [codegen id : 5] +Input [2]: [ss_store_sk#1, ss_net_profit#2] + +(10) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) +(11) CometFilter +Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Condition : isnotnull(s_store_sk#7) -(9) ColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] +(12) ColumnarToRow [codegen id : 4] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(13) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) CometFilter -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(12) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(14) CometFilter +Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_store_sk#10) -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] +(15) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : isnotnull(s_store_sk#13) +(16) CometFilter +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) -(15) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#13, s_state#14] +(17) CometBroadcastExchange +Input [2]: [s_store_sk#14, s_state#15] +Arguments: [s_store_sk#14, s_state#15] -(16) BroadcastExchange -Input [2]: [s_store_sk#13, s_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(18) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Right output [2]: [s_store_sk#14, s_state#15] +Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None +(19) CometProject +Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#15] +Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15], [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] -(18) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] -(19) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#15] +(21) CometBroadcastHashJoin +Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] +Right output [1]: [d_date_sk#16] +Arguments: [ss_sold_date_sk#12], [d_date_sk#16], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(22) CometProject +Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15, d_date_sk#16] +Arguments: [ss_net_profit#11, s_state#15], [ss_net_profit#11, s_state#15] + +(23) CometHashAggregate +Input [2]: [ss_net_profit#11, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] + +(24) ColumnarToRow [codegen id : 1] +Input [2]: [s_state#15, sum#17] -(21) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#14] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] - -(22) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#14] -Keys [1]: [s_state#14] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [s_state#14, sum#17] - -(23) Exchange -Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(24) HashAggregate [codegen id : 5] -Input [2]: [s_state#14, sum#17] -Keys [1]: [s_state#14] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] -Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] - -(25) Sort [codegen id : 5] -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 - -(26) Window -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] - -(27) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] +(25) Exchange +Input [2]: [s_state#15, sum#17] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(26) HashAggregate [codegen id : 2] +Input [2]: [s_state#15, sum#17] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#18] +Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#18,17,2) AS _w0#19, s_state#15] + +(27) Sort [codegen id : 2] +Input [3]: [s_state#15, _w0#19, s_state#15] +Arguments: [s_state#15 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 + +(28) Window +Input [3]: [s_state#15, _w0#19, s_state#15] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#15, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#15], [_w0#19 DESC NULLS LAST] + +(29) Filter [codegen id : 3] +Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] Condition : (ranking#20 <= 5) -(28) Project [codegen id : 6] -Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] +(30) Project [codegen id : 3] +Output [1]: [s_state#15] +Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] -(29) BroadcastExchange -Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] +(31) BroadcastExchange +Input [1]: [s_state#15] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(30) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [s_state#8] -Right keys [1]: [s_state#14] +(32) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [s_state#9] +Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(31) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(33) BroadcastExchange +Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(32) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] +Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(33) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_state#8, s_county#7] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] +(35) Project [codegen id : 5] +Output [3]: [ss_net_profit#2, s_state#9, s_county#8] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#9] -(34) Expand [codegen id : 8] -Input [3]: [ss_net_profit#2, s_state#8, s_county#7] -Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] +(36) Expand [codegen id : 5] +Input [3]: [ss_net_profit#2, s_state#9, s_county#8] +Arguments: [[ss_net_profit#2, s_state#9, s_county#8, 0], [ss_net_profit#2, s_state#9, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] -(35) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 5] Input [4]: [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#24] Results [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] -(36) Exchange +(38) Exchange Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] -Arguments: hashpartitioning(s_state#21, s_county#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(s_state#21, s_county#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(37) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 6] Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#26] Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS total_sum#27, s_state#21, s_county#22, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS lochierarchy#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS _w0#29, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS _w1#30, CASE WHEN (cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint) = 0) THEN s_state#21 END AS _w2#31] -(38) Exchange +(40) Exchange Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] -Arguments: hashpartitioning(_w1#30, _w2#31, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(_w1#30, _w2#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(39) Sort [codegen id : 10] +(41) Sort [codegen id : 7] Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] Arguments: [_w1#30 ASC NULLS FIRST, _w2#31 ASC NULLS FIRST, _w0#29 DESC NULLS LAST], false, 0 -(40) Window +(42) Window Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] Arguments: [rank(_w0#29) windowspecdefinition(_w1#30, _w2#31, _w0#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#32], [_w1#30, _w2#31], [_w0#29 DESC NULLS LAST] -(41) Project [codegen id : 11] +(43) Project [codegen id : 8] Output [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] Input [8]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31, rank_within_parent#32] -(42) TakeOrderedAndProject +(44) TakeOrderedAndProject Input [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN s_state#21 END ASC NULLS FIRST, rank_within_parent#32 ASC NULLS FIRST], [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(43) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#33] +(45) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#33] -Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1200)) AND (d_month_seq#33 <= 1211)) AND isnotnull(d_date_sk#5)) +(46) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(45) CometProject -Input [2]: [d_date_sk#5, d_month_seq#33] +(47) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index 0e01c5f710..f4a03a2d2f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (11) + WholeStageCodegen (8) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (10) + WholeStageCodegen (7) Sort [_w1,_w2,_w0] InputAdapter Exchange [_w1,_w2] #1 - WholeStageCodegen (9) + WholeStageCodegen (6) HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] InputAdapter Exchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] Expand [ss_net_profit,s_state,s_county] Project [ss_net_profit,s_state,s_county] BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -30,45 +30,42 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) + BroadcastExchange #5 + WholeStageCodegen (4) BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) + BroadcastExchange #6 + WholeStageCodegen (3) Project [s_state] Filter [ranking] InputAdapter Window [_w0,s_state] - WholeStageCodegen (5) + WholeStageCodegen (2) Sort [s_state,_w0] HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] InputAdapter - Exchange [s_state] #6 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + Exchange [s_state] #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [s_state,ss_net_profit] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #8 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 624103b664..f2128cfff1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -1,42 +1,41 @@ == Physical Plan == -* Sort (38) -+- Exchange (37) - +- * HashAggregate (36) - +- Exchange (35) - +- * HashAggregate (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildLeft (25) - : :- BroadcastExchange (5) - : : +- * ColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.item (1) - : +- Union (24) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (8) - : : : +- CometFilter (7) - : : : +- CometScan parquet spark_catalog.default.web_sales (6) - : : +- ReusedExchange (9) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * ColumnarToRow (14) - : : : +- CometFilter (13) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : +- ReusedExchange (15) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * ColumnarToRow (20) - : : +- CometFilter (19) - : : +- CometScan parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (21) - +- BroadcastExchange (31) - +- * ColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan parquet spark_catalog.default.time_dim (27) +* Sort (37) ++- Exchange (36) + +- * HashAggregate (35) + +- Exchange (34) + +- * ColumnarToRow (33) + +- CometHashAggregate (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometBroadcastExchange (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.item (1) + : +- CometUnion (23) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.web_sales (5) + : : +- CometBroadcastExchange (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.date_dim (7) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : +- ReusedExchange (15) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (19) + : : +- CometScan parquet spark_catalog.default.store_sales (18) + : +- ReusedExchange (20) + +- CometBroadcastExchange (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.time_dim (26) (1) Scan parquet spark_catalog.default.item @@ -54,14 +53,11 @@ Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull( Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] Arguments: [i_item_sk#1, i_brand_id#2, i_brand#3], [i_item_sk#1, i_brand_id#2, i_brand#3] -(4) ColumnarToRow [codegen id : 1] +(4) CometBroadcastExchange Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#3] +Arguments: [i_item_sk#1, i_brand_id#2, i_brand#3] -(5) BroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#3] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(6) Scan parquet spark_catalog.default.web_sales +(5) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -69,186 +65,189 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#8), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] Condition : (isnotnull(ws_item_sk#6) AND isnotnull(ws_sold_time_sk#5)) -(8) ColumnarToRow [codegen id : 3] -Input [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] +(7) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] +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 + +(8) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11)) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) + +(9) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Arguments: [d_date_sk#10], [d_date_sk#10] -(9) ReusedExchange [Reuses operator id: 43] -Output [1]: [d_date_sk#10] +(10) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [ws_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight -(11) Project [codegen id : 3] -Output [3]: [ws_ext_sales_price#7 AS ext_price#11, ws_item_sk#6 AS sold_item_sk#12, ws_sold_time_sk#5 AS time_sk#13] +(12) CometProject Input [5]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8, d_date_sk#10] +Arguments: [ext_price#13, sold_item_sk#14, time_sk#15], [ws_ext_sales_price#7 AS ext_price#13, ws_item_sk#6 AS sold_item_sk#14, ws_sold_time_sk#5 AS time_sk#15] -(12) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +(13) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_sold_time_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] ReadSchema: struct -(13) CometFilter -Input [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_item_sk#15) AND isnotnull(cs_sold_time_sk#14)) - -(14) ColumnarToRow [codegen id : 5] -Input [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +(14) CometFilter +Input [4]: [cs_sold_time_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_sold_time_sk#16)) -(15) ReusedExchange [Reuses operator id: 43] -Output [1]: [d_date_sk#19] +(15) ReusedExchange [Reuses operator id: 10] +Output [1]: [d_date_sk#21] -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [4]: [cs_sold_time_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Right output [1]: [d_date_sk#21] +Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight -(17) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#16 AS ext_price#20, cs_item_sk#15 AS sold_item_sk#21, cs_sold_time_sk#14 AS time_sk#22] -Input [5]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17, d_date_sk#19] +(17) CometProject +Input [5]: [cs_sold_time_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] +Arguments: [ext_price#22, sold_item_sk#23, time_sk#24], [cs_ext_sales_price#18 AS ext_price#22, cs_item_sk#17 AS sold_item_sk#23, cs_sold_time_sk#16 AS time_sk#24] (18) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct (19) CometFilter -Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) +Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_time_sk#25)) -(20) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +(20) ReusedExchange [Reuses operator id: 10] +Output [1]: [d_date_sk#30] -(21) ReusedExchange [Reuses operator id: 43] -Output [1]: [d_date_sk#28] +(21) CometBroadcastHashJoin +Left output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Right output [1]: [d_date_sk#30] +Arguments: [ss_sold_date_sk#28], [d_date_sk#30], Inner, BuildRight -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#28] -Join type: Inner -Join condition: None +(22) CometProject +Input [5]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30] +Arguments: [ext_price#31, sold_item_sk#32, time_sk#33], [ss_ext_sales_price#27 AS ext_price#31, ss_item_sk#26 AS sold_item_sk#32, ss_sold_time_sk#25 AS time_sk#33] -(23) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#25 AS ext_price#29, ss_item_sk#24 AS sold_item_sk#30, ss_sold_time_sk#23 AS time_sk#31] -Input [5]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#28] +(23) CometUnion +Child 0 Input [3]: [ext_price#13, sold_item_sk#14, time_sk#15] +Child 1 Input [3]: [ext_price#22, sold_item_sk#23, time_sk#24] +Child 2 Input [3]: [ext_price#31, sold_item_sk#32, time_sk#33] -(24) Union +(24) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand_id#2, i_brand#3] +Right output [3]: [ext_price#13, sold_item_sk#14, time_sk#15] +Arguments: [i_item_sk#1], [sold_item_sk#14], Inner, BuildLeft -(25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [sold_item_sk#12] -Join type: Inner -Join condition: None +(25) CometProject +Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#13, sold_item_sk#14, time_sk#15] +Arguments: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15], [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15] -(26) Project [codegen id : 9] -Output [4]: [i_brand_id#2, i_brand#3, ext_price#11, time_sk#13] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#11, sold_item_sk#12, time_sk#13] - -(27) Scan parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +(26) Scan parquet spark_catalog.default.time_dim +Output [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) - -(29) CometProject -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Arguments: [t_time_sk#32, t_hour#33, t_minute#34], [t_time_sk#32, t_hour#33, t_minute#34] - -(30) ColumnarToRow [codegen id : 8] -Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] - -(31) BroadcastExchange -Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [time_sk#13] -Right keys [1]: [t_time_sk#32] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#3, ext_price#11, t_hour#33, t_minute#34] -Input [7]: [i_brand_id#2, i_brand#3, ext_price#11, time_sk#13, t_time_sk#32, t_hour#33, t_minute#34] - -(34) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#3, ext_price#11, t_hour#33, t_minute#34] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] -Functions [1]: [partial_sum(UnscaledValue(ext_price#11))] -Aggregate Attributes [1]: [sum#36] -Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] - -(35) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(36) HashAggregate [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] -Functions [1]: [sum(UnscaledValue(ext_price#11))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#11))#38] -Results [5]: [i_brand_id#2 AS brand_id#39, i_brand#3 AS brand#40, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#11))#38,17,2) AS ext_price#41] - -(37) Exchange -Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] -Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(38) Sort [codegen id : 11] -Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] -Arguments: [ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST], true, 0 +(27) CometFilter +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Condition : (((t_meal_time#37 = breakfast ) OR (t_meal_time#37 = dinner )) AND isnotnull(t_time_sk#34)) + +(28) CometProject +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Arguments: [t_time_sk#34, t_hour#35, t_minute#36], [t_time_sk#34, t_hour#35, t_minute#36] + +(29) CometBroadcastExchange +Input [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: [t_time_sk#34, t_hour#35, t_minute#36] + +(30) CometBroadcastHashJoin +Left output [4]: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15] +Right output [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: [time_sk#15], [t_time_sk#34], Inner, BuildRight + +(31) CometProject +Input [7]: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15, t_time_sk#34, t_hour#35, t_minute#36] +Arguments: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36], [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36] + +(32) CometHashAggregate +Input [5]: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] +Functions [1]: [partial_sum(UnscaledValue(ext_price#13))] + +(33) ColumnarToRow [codegen id : 1] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#38] + +(34) Exchange +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#38] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(35) HashAggregate [codegen id : 2] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#38] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] +Functions [1]: [sum(UnscaledValue(ext_price#13))] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#13))#39] +Results [5]: [i_brand_id#2 AS brand_id#40, i_brand#3 AS brand#41, t_hour#35, t_minute#36, MakeDecimal(sum(UnscaledValue(ext_price#13))#39,17,2) AS ext_price#42] + +(36) Exchange +Input [5]: [brand_id#40, brand#41, t_hour#35, t_minute#36, ext_price#42] +Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(37) Sort [codegen id : 3] +Input [5]: [brand_id#40, brand#41, t_hour#35, t_minute#36, ext_price#42] +Arguments: [ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (43) -+- * ColumnarToRow (42) - +- CometProject (41) - +- CometFilter (40) - +- CometScan parquet spark_catalog.default.date_dim (39) +Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (42) ++- * ColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.date_dim (38) -(39) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#42, d_moy#43] +(38) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] 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 -(40) CometFilter -Input [3]: [d_date_sk#10, d_year#42, d_moy#43] -Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 11)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#10)) +(39) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11)) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) -(41) CometProject -Input [3]: [d_date_sk#10, d_year#42, d_moy#43] +(40) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(42) ColumnarToRow [codegen id : 1] +(41) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(43) BroadcastExchange +(42) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index bea5376a00..fd6777886a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -1,69 +1,53 @@ -WholeStageCodegen (11) +WholeStageCodegen (3) Sort [ext_price,brand_id] InputAdapter Exchange [ext_price,brand_id] #1 - WholeStageCodegen (10) + WholeStageCodegen (2) HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] InputAdapter Exchange [i_brand,i_brand_id,t_hour,t_minute] #2 - WholeStageCodegen (9) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] - BroadcastHashJoin [time_sk,t_time_sk] - Project [i_brand_id,i_brand,ext_price,time_sk] - BroadcastHashJoin [i_item_sk,sold_item_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_brand] - CometFilter [i_manager_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - InputAdapter - Union - WholeStageCodegen (3) - Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_item_sk,ws_sold_time_sk] - CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_moy,d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (5) - Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_sold_time_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (7) - Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_sold_time_sk] - CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_meal_time,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] + CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] + CometBroadcastHashJoin [time_sk,t_time_sk] + CometProject [i_brand_id,i_brand,ext_price,time_sk] + CometBroadcastHashJoin [i_item_sk,sold_item_sk] + CometBroadcastExchange #3 + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometUnion + CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_sold_time_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometFilter [cs_item_sk,cs_sold_time_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_sold_time_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #6 + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_meal_time,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index 4e5d9e9f60..26edd145c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -1,74 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * SortMergeJoin LeftOuter (65) - :- * Sort (58) - : +- Exchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * ColumnarToRow (3) - : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * ColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * ColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * ColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * ColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * ColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * ColumnarToRow (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.promotion (51) - +- * Sort (64) - +- Exchange (63) - +- * ColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.catalog_returns (59) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- * Project (63) + +- * SortMergeJoin LeftOuter (62) + :- * Sort (55) + : +- Exchange (54) + : +- * Project (53) + : +- * BroadcastHashJoin LeftOuter BuildRight (52) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * ColumnarToRow (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (35) + : : : : +- CometBroadcastHashJoin (34) + : : : : :- CometProject (29) + : : : : : +- CometBroadcastHashJoin (28) + : : : : : :- CometProject (23) + : : : : : : +- CometBroadcastHashJoin (22) + : : : : : : :- CometProject (17) + : : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : : :- CometProject (12) + : : : : : : : : +- CometBroadcastHashJoin (11) + : : : : : : : : :- CometProject (7) + : : : : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : : : : :- CometFilter (2) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- CometBroadcastExchange (5) + : : : : : : : : : +- CometFilter (4) + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (3) + : : : : : : : : +- CometBroadcastExchange (10) + : : : : : : : : +- CometFilter (9) + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (8) + : : : : : : : +- CometBroadcastExchange (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : : +- CometBroadcastExchange (21) + : : : : : : +- CometProject (20) + : : : : : : +- CometFilter (19) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) + : : : : : +- CometBroadcastExchange (27) + : : : : : +- CometProject (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan parquet spark_catalog.default.household_demographics (24) + : : : : +- CometBroadcastExchange (33) + : : : : +- CometProject (32) + : : : : +- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.date_dim (30) + : : : +- CometBroadcastExchange (38) + : : : +- CometFilter (37) + : : : +- CometScan parquet spark_catalog.default.date_dim (36) + : : +- BroadcastExchange (45) + : : +- * ColumnarToRow (44) + : : +- CometFilter (43) + : : +- CometScan parquet spark_catalog.default.date_dim (42) + : +- BroadcastExchange (51) + : +- * ColumnarToRow (50) + : +- CometFilter (49) + : +- CometScan parquet spark_catalog.default.promotion (48) + +- * Sort (61) + +- Exchange (60) + +- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.catalog_returns (56) (1) Scan parquet spark_catalog.default.catalog_sales @@ -83,10 +80,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -(7) BroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) +(6) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] +Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +(7) CometProject Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -(10) Scan parquet spark_catalog.default.warehouse +(8) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [w_warehouse_sk#14, w_warehouse_name#15] -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +(12) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#16, i_item_desc#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#16, i_item_desc#17] - -(19) BroadcastExchange +(15) CometBroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#16, i_item_desc#17] -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +Right output [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(17) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(22) Scan parquet spark_catalog.default.customer_demographics +(18) Scan parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#18, cd_marital_status#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_marital_status), EqualTo(cd_marital_status,D), IsNotNull(cd_demo_sk)] ReadSchema: struct -(23) CometFilter +(19) CometFilter Input [2]: [cd_demo_sk#18, cd_marital_status#19] Condition : ((isnotnull(cd_marital_status#19) AND (cd_marital_status#19 = D)) AND isnotnull(cd_demo_sk#18)) -(24) CometProject +(20) CometProject Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) ColumnarToRow [codegen id : 4] +(21) CometBroadcastExchange Input [1]: [cd_demo_sk#18] +Arguments: [cd_demo_sk#18] -(26) BroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [cd_demo_sk#18] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(23) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] +Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(29) Scan parquet spark_catalog.default.household_demographics +(24) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#20, hd_buy_potential#21] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct -(30) CometFilter +(25) CometFilter Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Condition : ((isnotnull(hd_buy_potential#21) AND (hd_buy_potential#21 = >10000 )) AND isnotnull(hd_demo_sk#20)) -(31) CometProject +(26) CometProject Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) ColumnarToRow [codegen id : 5] +(27) CometBroadcastExchange Input [1]: [hd_demo_sk#20] +Arguments: [hd_demo_sk#20] -(33) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(28) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [hd_demo_sk#20] +Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(29) CometProject Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +(30) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +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 -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(31) CometFilter +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)) -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +(32) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] + +(33) CometBroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] + +(34) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight + +(35) CometProject Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(39) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] +(36) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_week_seq#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) +(37) CometFilter +Input [2]: [d_date_sk#26, d_week_seq#27] +Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) -(41) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] +(38) CometBroadcastExchange +Input [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_date_sk#26, d_week_seq#27] -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] +(39) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +Right output [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None +(40) CometProject +Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] +(41) ColumnarToRow [codegen id : 3] +Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(45) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] +(42) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_date#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) +(43) CometFilter +Input [2]: [d_date_sk#28, d_date#29] +Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) -(47) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] +(44) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_date#29] -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(45) BroadcastExchange +Input [2]: [d_date_sk#28, d_date#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(49) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#28] Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) +Join condition: (d_date#29 > date_add(d_date#23, 5)) -(50) Project [codegen id : 10] +(47) Project [codegen id : 3] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] +Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -(51) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] +(48) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) +(49) CometFilter +Input [1]: [p_promo_sk#30] +Condition : isnotnull(p_promo_sk#30) -(53) ColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] +(50) ColumnarToRow [codegen id : 2] +Input [1]: [p_promo_sk#30] -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(51) BroadcastExchange +Input [1]: [p_promo_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(55) BroadcastHashJoin [codegen id : 10] +(52) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] +Right keys [1]: [p_promo_sk#30] Join type: LeftOuter Join condition: None -(56) Project [codegen id : 10] +(53) Project [codegen id : 3] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -(57) Exchange +(54) Exchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(58) Sort [codegen id : 11] +(55) Sort [codegen id : 4] Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 -(59) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +(56) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) +(57) CometFilter +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] +(58) CometProject +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] -(62) ColumnarToRow [codegen id : 12] -Input [2]: [cr_item_sk#30, cr_order_number#31] +(59) ColumnarToRow [codegen id : 5] +Input [2]: [cr_item_sk#31, cr_order_number#32] -(63) Exchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(60) Exchange +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(64) Sort [codegen id : 13] -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST], false, 0 +(61) Sort [codegen id : 6] +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 14] +(62) SortMergeJoin [codegen id : 7] Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#30, cr_order_number#31] +Right keys [2]: [cr_item_sk#31, cr_order_number#32] Join type: LeftOuter Join condition: None -(66) Project [codegen id : 14] +(63) Project [codegen id : 7] Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -(67) HashAggregate [codegen id : 14] +(64) HashAggregate [codegen id : 7] Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Aggregate Attributes [1]: [count#34] +Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] -(68) Exchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(65) Exchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(69) HashAggregate [codegen id : 15] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +(66) HashAggregate [codegen id : 8] +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#35 AS no_promo#36, count(1)#35 AS promo#37, count(1)#35 AS total_cnt#38] +Aggregate Attributes [1]: [count(1)#36] +Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#36 AS no_promo#37, count(1)#36 AS promo#38, count(1)#36 AS total_cnt#39] -(70) TakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] -Arguments: 100, [total_cnt#38 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] +(67) TakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#37, promo#38, total_cnt#39] +Arguments: 100, [total_cnt#39 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#37, promo#38, total_cnt#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(71) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +(68) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] 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 -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] -Condition : ((((isnotnull(d_year#39) AND (d_year#39 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(69) CometFilter +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)) -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +(70) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(75) BroadcastExchange +(72) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt index 17fc9dee7e..bea1fd4a16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt @@ -1,38 +1,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - WholeStageCodegen (15) + WholeStageCodegen (8) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - WholeStageCodegen (14) + WholeStageCodegen (7) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] Project [w_warehouse_name,i_item_desc,d_week_seq] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (4) Sort [cs_item_sk,cs_order_number] InputAdapter Exchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) + WholeStageCodegen (3) 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] BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -43,72 +43,50 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_year,d_date_sk,d_week_seq,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [d_week_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange #4 + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange #7 + CometProject [cd_demo_sk] + CometFilter [cd_marital_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange #8 + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange #10 + CometFilter [d_week_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) + BroadcastExchange #11 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) + BroadcastExchange #12 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (6) Sort [cr_item_sk,cr_order_number] InputAdapter - Exchange [cr_item_sk,cr_order_number] #12 - WholeStageCodegen (12) + Exchange [cr_item_sk,cr_order_number] #13 + WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index ef2c38aea4..88dcba0fb4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt @@ -1,36 +1,37 @@ == Physical Plan == -* Sort (32) -+- Exchange (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (24) - : +- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (28) - +- * ColumnarToRow (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.customer (25) +* Sort (33) ++- Exchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Filter (25) + : +- * HashAggregate (24) + : +- Exchange (23) + : +- * ColumnarToRow (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.household_demographics (15) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.customer (26) (1) Scan parquet spark_catalog.default.store_sales @@ -45,174 +46,179 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#7] +(4) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(7) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +(8) CometProject Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#10, s_county#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : (s_county#9 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] +(10) CometFilter +Input [2]: [s_store_sk#10, s_county#11] +Condition : (s_county#11 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#10)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] +(11) CometProject +Input [2]: [s_store_sk#10, s_county#11] +Arguments: [s_store_sk#10], [s_store_sk#10] -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#10] +Arguments: [s_store_sk#10] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +Right output [1]: [s_store_sk#10] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] +(14) CometProject +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -(14) Scan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +(15) Scan parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.0) END) AND isnotnull(hd_demo_sk#10)) +(16) CometFilter +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Condition : ((((isnotnull(hd_vehicle_count#15) AND ((hd_buy_potential#13 = >10000 ) OR (hd_buy_potential#13 = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN ((cast(hd_dep_count#14 as double) / cast(hd_vehicle_count#15 as double)) > 1.0) END) AND isnotnull(hd_demo_sk#12)) -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] +(17) CometProject +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Arguments: [hd_demo_sk#12], [hd_demo_sk#12] -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#12] +Arguments: [hd_demo_sk#12] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(19) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Right output [1]: [hd_demo_sk#12] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None +(20) CometProject +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] +Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] +(21) CometHashAggregate Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) Exchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) ColumnarToRow [codegen id : 1] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] + +(23) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(23) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +(24) HashAggregate [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] +Aggregate Attributes [1]: [count(1)#17] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(24) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) +(25) Filter [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18] +Condition : ((cnt#18 >= 1) AND (cnt#18 <= 5)) -(25) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(26) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) +(27) CometFilter +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Condition : isnotnull(c_customer_sk#19) -(27) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(28) ColumnarToRow [codegen id : 2] +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(28) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(29) BroadcastExchange +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(29) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#19] Join type: Inner Join condition: None -(30) Project [codegen id : 6] -Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(31) Project [codegen id : 3] +Output [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18, c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(31) Exchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(32) Exchange +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: rangepartitioning(cnt#18 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 7] -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [cnt#17 DESC NULLS LAST], true, 0 +(33) Sort [codegen id : 4] +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: [cnt#18 DESC NULLS LAST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(33) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +(34) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_dom#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] -Condition : ((((isnotnull(d_dom#24) AND (d_dom#24 >= 1)) AND (d_dom#24 <= 2)) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(35) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(35) CometProject -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +(36) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(37) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index 7c5ee1ef5a..060c3e1530 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -1,24 +1,24 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) Sort [cnt] InputAdapter Exchange [cnt] #1 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [cnt] HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter Exchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,27 +29,21 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange #5 + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange #6 + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index dad94eb2cd..83e2737fd0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -2,74 +2,74 @@ TakeOrderedAndProject (71) +- * Project (70) +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * ColumnarToRow (36) - : : : +- CometFilter (35) - : : : +- CometScan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (17) + : : : +- * HashAggregate (16) + : : : +- Exchange (15) + : : : +- * ColumnarToRow (14) + : : : +- CometHashAggregate (13) + : : : +- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- BroadcastExchange (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * ColumnarToRow (31) + : : +- CometHashAggregate (30) + : : +- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- CometBroadcastExchange (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- BroadcastExchange (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * ColumnarToRow (47) + : +- CometHashAggregate (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometProject (42) + : : +- CometBroadcastHashJoin (41) + : : :- CometFilter (37) + : : : +- CometScan parquet spark_catalog.default.customer (36) + : : +- CometBroadcastExchange (40) + : : +- CometFilter (39) + : : +- CometScan parquet spark_catalog.default.web_sales (38) + : +- ReusedExchange (43) +- BroadcastExchange (68) +- * HashAggregate (67) +- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * ColumnarToRow (55) - : : +- CometFilter (54) - : : +- CometScan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) + +- * ColumnarToRow (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometBroadcastHashJoin (62) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (61) (1) Scan parquet spark_catalog.default.customer @@ -83,10 +83,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] Condition : isnotnull(ss_customer_sk#5) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -(7) BroadcastExchange -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Right output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: [c_customer_sk#1], [ss_customer_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +(7) CometProject Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 75] +(8) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#9, d_year#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#9, d_year#10] +Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [2]: [d_date_sk#9, d_year#10] +Arguments: [d_date_sk#9, d_year#10] -(12) Project [codegen id : 3] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] +(11) CometBroadcastHashJoin +Left output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#9, d_year#10] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight + +(12) CometProject Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] -(13) HashAggregate [codegen id : 3] +(13) CometHashAggregate Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum#11] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -(14) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(14) ColumnarToRow [codegen id : 1] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#11] + +(15) Exchange +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#11] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 16] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] +(16) HashAggregate [codegen id : 8] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#11] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] -Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#12] +Results [2]: [c_customer_id#2 AS customer_id#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#12,17,2) AS year_total#14] -(16) Filter [codegen id : 16] -Input [2]: [customer_id#14, year_total#15] -Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) +(17) Filter [codegen id : 8] +Input [2]: [customer_id#13, year_total#14] +Condition : (isnotnull(year_total#14) AND (year_total#14 > 0.00)) -(17) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +(18) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter -Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] -Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) - -(19) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +(19) CometFilter +Input [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_customer_id#16)) (20) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (21) CometFilter -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#20) +Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Condition : isnotnull(ss_customer_sk#19) -(22) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +(22) CometBroadcastExchange +Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Arguments: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -(23) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] +Right output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Arguments: [c_customer_sk#15], [ss_customer_sk#19], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#16] -Right keys [1]: [ss_customer_sk#20] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] -Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] - -(26) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#24, d_year#25] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(24) CometProject +Input [7]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18, ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Arguments: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21], [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21] -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] - -(29) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] -Aggregate Attributes [1]: [sum#26] -Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] - -(30) Exchange -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] -Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] -Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] - -(32) BroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#23, d_year#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct -(33) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#28] +(26) CometFilter +Input [2]: [d_date_sk#23, d_year#24] +Condition : (((isnotnull(d_year#24) AND (d_year#24 = 2002)) AND d_year#24 IN (2001,2002)) AND isnotnull(d_date_sk#23)) + +(27) CometBroadcastExchange +Input [2]: [d_date_sk#23, d_year#24] +Arguments: [d_date_sk#23, d_year#24] + +(28) CometBroadcastHashJoin +Left output [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21] +Right output [2]: [d_date_sk#23, d_year#24] +Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight + +(29) CometProject +Input [7]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21, d_date_sk#23, d_year#24] +Arguments: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24], [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] + +(30) CometHashAggregate +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] +Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#20))] + +(31) ColumnarToRow [codegen id : 2] +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#25] + +(32) Exchange +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#25] +Arguments: hashpartitioning(c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(33) HashAggregate [codegen id : 3] +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#25] +Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] +Functions [1]: [sum(UnscaledValue(ss_net_paid#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#20))#12] +Results [4]: [c_customer_id#16 AS customer_id#26, c_first_name#17 AS customer_first_name#27, c_last_name#18 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#20))#12,17,2) AS year_total#29] + +(34) BroadcastExchange +Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(35) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#26] Join type: Inner Join condition: None -(34) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +(36) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#30, c_customer_id#31, c_first_name#32, c_last_name#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) - -(36) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +(37) CometFilter +Input [4]: [c_customer_sk#30, c_customer_id#31, c_first_name#32, c_last_name#33] +Condition : (isnotnull(c_customer_sk#30) AND isnotnull(c_customer_id#31)) -(37) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(38) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_bill_customer_sk#36) +(39) CometFilter +Input [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_bill_customer_sk#34) -(39) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(40) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +Arguments: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] -(40) BroadcastExchange -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(41) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#30, c_customer_id#31, c_first_name#32, c_last_name#33] +Right output [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +Arguments: [c_customer_sk#30], [ws_bill_customer_sk#34], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#32] -Right keys [1]: [ws_bill_customer_sk#36] -Join type: Inner -Join condition: None +(42) CometProject +Input [7]: [c_customer_sk#30, c_customer_id#31, c_first_name#32, c_last_name#33, ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +Arguments: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36], [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36] -(42) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] -Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(43) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#38, d_year#39] -(43) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#40, d_year#41] +(44) CometBroadcastHashJoin +Left output [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36] +Right output [2]: [d_date_sk#38, d_year#39] +Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None +(45) CometProject +Input [7]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39] +Arguments: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39], [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39] + +(46) CometHashAggregate +Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39] +Keys [4]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#35))] + +(47) ColumnarToRow [codegen id : 4] +Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] -(45) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] - -(46) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] -Aggregate Attributes [1]: [sum#42] -Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] - -(47) Exchange -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(48) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] -Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] -Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] - -(49) Filter [codegen id : 11] -Input [2]: [customer_id#45, year_total#46] -Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) - -(50) BroadcastExchange -Input [2]: [customer_id#45, year_total#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#45] +(48) Exchange +Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] +Arguments: hashpartitioning(c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(49) HashAggregate [codegen id : 5] +Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] +Keys [4]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39] +Functions [1]: [sum(UnscaledValue(ws_net_paid#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#35))#41] +Results [2]: [c_customer_id#31 AS customer_id#42, MakeDecimal(sum(UnscaledValue(ws_net_paid#35))#41,17,2) AS year_total#43] + +(50) Filter [codegen id : 5] +Input [2]: [customer_id#42, year_total#43] +Condition : (isnotnull(year_total#43) AND (year_total#43 > 0.00)) + +(51) BroadcastExchange +Input [2]: [customer_id#42, year_total#43] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(52) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#42] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] -Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] +(53) Project [codegen id : 8] +Output [7]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#43] +Input [8]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#42, year_total#43] -(53) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +(54) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter -Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] -Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) - -(55) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +(55) CometFilter +Input [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] +Condition : (isnotnull(c_customer_sk#44) AND isnotnull(c_customer_id#45)) (56) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#51)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (57) CometFilter -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_bill_customer_sk#51) +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +Condition : isnotnull(ws_bill_customer_sk#48) -(58) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +(58) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +Arguments: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] -(59) BroadcastExchange -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(59) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] +Right output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +Arguments: [c_customer_sk#44], [ws_bill_customer_sk#48], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#47] -Right keys [1]: [ws_bill_customer_sk#51] -Join type: Inner -Join condition: None +(60) CometProject +Input [7]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +Arguments: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50], [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50] -(61) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] -Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +(61) ReusedExchange [Reuses operator id: 27] +Output [2]: [d_date_sk#52, d_year#53] -(62) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#55, d_year#56] +(62) CometBroadcastHashJoin +Left output [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50] +Right output [2]: [d_date_sk#52, d_year#53] +Arguments: [ws_sold_date_sk#50], [d_date_sk#52], Inner, BuildRight -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#53] -Right keys [1]: [d_date_sk#55] -Join type: Inner -Join condition: None +(63) CometProject +Input [7]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50, d_date_sk#52, d_year#53] +Arguments: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53], [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53] -(64) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] +(64) CometHashAggregate +Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53] +Keys [4]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#49))] -(65) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] -Aggregate Attributes [1]: [sum#57] -Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +(65) ColumnarToRow [codegen id : 6] +Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] (66) Exchange -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] +Arguments: hashpartitioning(c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(67) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] -Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#52))#44] -Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#52))#44,17,2) AS year_total#60] +(67) HashAggregate [codegen id : 7] +Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] +Keys [4]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53] +Functions [1]: [sum(UnscaledValue(ws_net_paid#49))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#49))#41] +Results [2]: [c_customer_id#45 AS customer_id#55, MakeDecimal(sum(UnscaledValue(ws_net_paid#49))#41,17,2) AS year_total#56] (68) BroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +Input [2]: [customer_id#55, year_total#56] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(69) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#59] +(69) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#55] Join type: Inner -Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) +Join condition: (CASE WHEN (year_total#43 > 0.00) THEN (year_total#56 / year_total#43) END > CASE WHEN (year_total#14 > 0.00) THEN (year_total#29 / year_total#14) END) -(70) Project [codegen id : 16] -Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] +(70) Project [codegen id : 8] +Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] +Input [9]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#43, customer_id#55, year_total#56] (71) TakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] +Arguments: 100, [customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (75) +- * ColumnarToRow (74) +- CometFilter (73) @@ -443,9 +439,9 @@ Input [2]: [d_date_sk#9, d_year#10] (75) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 BroadcastExchange (79) +- * ColumnarToRow (78) +- CometFilter (77) @@ -453,25 +449,25 @@ BroadcastExchange (79) (76) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +Output [2]: [d_date_sk#23, d_year#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (77) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) +Input [2]: [d_date_sk#23, d_year#24] +Condition : (((isnotnull(d_year#24) AND (d_year#24 = 2002)) AND d_year#24 IN (2001,2002)) AND isnotnull(d_date_sk#23)) (78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#25] +Input [2]: [d_date_sk#23, d_year#24] (79) BroadcastExchange -Input [2]: [d_date_sk#24, d_year#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Input [2]: [d_date_sk#23, d_year#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#22 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt index 9d3ae8fbee..2e6286d93b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - WholeStageCodegen (16) + WholeStageCodegen (8) Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] @@ -9,114 +9,98 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange #7 CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #13 + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index 5b9b2c2280..1f736c62cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -1,133 +1,136 @@ == Physical Plan == -TakeOrderedAndProject (129) -+- * Project (128) - +- * SortMergeJoin Inner (127) - :- * Sort (71) - : +- Exchange (70) - : +- * Filter (69) - : +- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- Union (62) - : :- * Project (23) - : : +- * SortMergeJoin LeftOuter (22) - : : :- * Sort (15) - : : : +- Exchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- * Sort (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : :- * Project (42) - : : +- * SortMergeJoin LeftOuter (41) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : :- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * ColumnarToRow (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) - : : : : +- ReusedExchange (27) - : : : +- ReusedExchange (30) - : : +- * Sort (40) - : : +- Exchange (39) - : : +- * ColumnarToRow (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometScan parquet spark_catalog.default.store_returns (35) - : +- * Project (61) - : +- * SortMergeJoin LeftOuter (60) - : :- * Sort (53) - : : +- Exchange (52) - : : +- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * ColumnarToRow (45) - : : : : +- CometFilter (44) - : : : : +- CometScan parquet spark_catalog.default.web_sales (43) - : : : +- ReusedExchange (46) - : : +- ReusedExchange (49) - : +- * Sort (59) - : +- Exchange (58) - : +- * ColumnarToRow (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan parquet spark_catalog.default.web_returns (54) - +- * Sort (126) - +- Exchange (125) - +- * Filter (124) - +- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- * HashAggregate (120) - +- Exchange (119) - +- * HashAggregate (118) - +- Union (117) - :- * Project (86) - : +- * SortMergeJoin LeftOuter (85) - : :- * Sort (82) - : : +- Exchange (81) - : : +- * Project (80) - : : +- * BroadcastHashJoin Inner BuildRight (79) - : : :- * Project (77) - : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : :- * ColumnarToRow (74) - : : : : +- CometFilter (73) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) - : : : +- ReusedExchange (75) - : : +- ReusedExchange (78) - : +- * Sort (84) - : +- ReusedExchange (83) - :- * Project (101) - : +- * SortMergeJoin LeftOuter (100) - : :- * Sort (97) - : : +- Exchange (96) - : : +- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * ColumnarToRow (89) - : : : : +- CometFilter (88) - : : : : +- CometScan parquet spark_catalog.default.store_sales (87) - : : : +- ReusedExchange (90) - : : +- ReusedExchange (93) - : +- * Sort (99) - : +- ReusedExchange (98) - +- * Project (116) - +- * SortMergeJoin LeftOuter (115) - :- * Sort (112) - : +- Exchange (111) - : +- * Project (110) - : +- * BroadcastHashJoin Inner BuildRight (109) - : :- * Project (107) - : : +- * BroadcastHashJoin Inner BuildRight (106) - : : :- * ColumnarToRow (104) - : : : +- CometFilter (103) - : : : +- CometScan parquet spark_catalog.default.web_sales (102) - : : +- ReusedExchange (105) - : +- ReusedExchange (108) - +- * Sort (114) - +- ReusedExchange (113) +TakeOrderedAndProject (132) ++- * Project (131) + +- * SortMergeJoin Inner (130) + :- * Sort (72) + : +- Exchange (71) + : +- * Filter (70) + : +- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- * HashAggregate (66) + : +- Exchange (65) + : +- * HashAggregate (64) + : +- Union (63) + : :- * Project (24) + : : +- * SortMergeJoin LeftOuter (23) + : : :- * Sort (16) + : : : +- Exchange (15) + : : : +- * ColumnarToRow (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.date_dim (9) + : : +- * Sort (22) + : : +- Exchange (21) + : : +- * ColumnarToRow (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : :- * Project (43) + : : +- * SortMergeJoin LeftOuter (42) + : : :- * Sort (35) + : : : +- Exchange (34) + : : : +- * ColumnarToRow (33) + : : : +- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometFilter (26) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (25) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- * Sort (41) + : : +- Exchange (40) + : : +- * ColumnarToRow (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.store_returns (36) + : +- * Project (62) + : +- * SortMergeJoin LeftOuter (61) + : :- * Sort (54) + : : +- Exchange (53) + : : +- * ColumnarToRow (52) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometProject (48) + : : : +- CometBroadcastHashJoin (47) + : : : :- CometFilter (45) + : : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- * Sort (60) + : +- Exchange (59) + : +- * ColumnarToRow (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.web_returns (55) + +- * Sort (129) + +- Exchange (128) + +- * Filter (127) + +- * HashAggregate (126) + +- Exchange (125) + +- * HashAggregate (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- Union (120) + :- * Project (89) + : +- * SortMergeJoin LeftOuter (88) + : :- * Sort (85) + : : +- Exchange (84) + : : +- * ColumnarToRow (83) + : : +- CometProject (82) + : : +- CometBroadcastHashJoin (81) + : : :- CometProject (77) + : : : +- CometBroadcastHashJoin (76) + : : : :- CometFilter (74) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) + : : : +- ReusedExchange (75) + : : +- CometBroadcastExchange (80) + : : +- CometFilter (79) + : : +- CometScan parquet spark_catalog.default.date_dim (78) + : +- * Sort (87) + : +- ReusedExchange (86) + :- * Project (104) + : +- * SortMergeJoin LeftOuter (103) + : :- * Sort (100) + : : +- Exchange (99) + : : +- * ColumnarToRow (98) + : : +- CometProject (97) + : : +- CometBroadcastHashJoin (96) + : : :- CometProject (94) + : : : +- CometBroadcastHashJoin (93) + : : : :- CometFilter (91) + : : : : +- CometScan parquet spark_catalog.default.store_sales (90) + : : : +- ReusedExchange (92) + : : +- ReusedExchange (95) + : +- * Sort (102) + : +- ReusedExchange (101) + +- * Project (119) + +- * SortMergeJoin LeftOuter (118) + :- * Sort (115) + : +- Exchange (114) + : +- * ColumnarToRow (113) + : +- CometProject (112) + : +- CometBroadcastHashJoin (111) + : :- CometProject (109) + : : +- CometBroadcastHashJoin (108) + : : :- CometFilter (106) + : : : +- CometScan parquet spark_catalog.default.web_sales (105) + : : +- ReusedExchange (107) + : +- ReusedExchange (110) + +- * Sort (117) + +- ReusedExchange (116) (1) Scan parquet spark_catalog.default.catalog_sales @@ -142,99 +145,106 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) ColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(8) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +(8) CometProject Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 133] +(9) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +(11) CometBroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: [d_date_sk#13, d_year#14] + +(12) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Right output [2]: [d_date_sk#13, d_year#14] +Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight + +(13) CometProject Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] + +(14) ColumnarToRow [codegen id : 1] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -(14) Exchange +(15) Exchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) Sort [codegen id : 4] +(16) Sort [codegen id : 2] Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 -(16) Scan parquet spark_catalog.default.catalog_returns +(17) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(17) CometFilter +(18) CometFilter Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) -(18) CometProject +(19) CometProject Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(19) ColumnarToRow [codegen id : 5] +(20) ColumnarToRow [codegen id : 3] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(20) Exchange +(21) Exchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) Sort [codegen id : 6] +(22) Sort [codegen id : 4] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] Arguments: [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 7] +(23) SortMergeJoin [codegen id : 5] Left keys [2]: [cs_order_number#2, cs_item_sk#1] Right keys [2]: [cr_order_number#16, cr_item_sk#15] Join type: LeftOuter Join condition: None -(23) Project [codegen id : 7] +(24) Project [codegen id : 5] Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(24) Scan parquet spark_catalog.default.store_sales +(25) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] @@ -242,84 +252,82 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(25) CometFilter +(26) CometFilter Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#22) -(26) ColumnarToRow [codegen id : 10] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(27) ReusedExchange [Reuses operator id: 8] +(27) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(28) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#22] -Right keys [1]: [i_item_sk#28] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight -(29) Project [codegen id : 10] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +(29) CometProject Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(30) ReusedExchange [Reuses operator id: 133] +(30) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#33, d_year#34] -(31) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#33] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Right output [2]: [d_date_sk#33, d_year#34] +Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight -(32) Project [codegen id : 10] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +(32) CometProject Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] + +(33) ColumnarToRow [codegen id : 6] +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -(33) Exchange +(34) Exchange Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(34) Sort [codegen id : 11] +(35) Sort [codegen id : 7] Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Arguments: [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST], false, 0 -(35) Scan parquet spark_catalog.default.store_returns +(36) Scan parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(36) CometFilter +(37) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(37) CometProject +(38) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(38) ColumnarToRow [codegen id : 12] +(39) ColumnarToRow [codegen id : 8] Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(39) Exchange +(40) Exchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(40) Sort [codegen id : 13] +(41) Sort [codegen id : 9] Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST], false, 0 -(41) SortMergeJoin [codegen id : 14] +(42) SortMergeJoin [codegen id : 10] Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] Join type: LeftOuter Join condition: None -(42) Project [codegen id : 14] +(43) Project [codegen id : 10] Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(43) Scan parquet spark_catalog.default.web_sales +(44) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] @@ -327,134 +335,132 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(44) CometFilter +(45) CometFilter Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Condition : isnotnull(ws_item_sk#42) -(45) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] - -(46) ReusedExchange [Reuses operator id: 8] +(46) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(47) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#42] -Right keys [1]: [i_item_sk#48] -Join type: Inner -Join condition: None +(47) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight -(48) Project [codegen id : 17] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +(48) CometProject Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(49) ReusedExchange [Reuses operator id: 133] +(49) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#46] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(50) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight -(51) Project [codegen id : 17] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +(51) CometProject Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(52) Exchange +(52) ColumnarToRow [codegen id : 11] Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(53) Sort [codegen id : 18] +(53) Exchange +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(54) Sort [codegen id : 12] Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Arguments: [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST], false, 0 -(54) Scan parquet spark_catalog.default.web_returns +(55) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(55) CometFilter +(56) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(56) CometProject +(57) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(57) ColumnarToRow [codegen id : 19] +(58) ColumnarToRow [codegen id : 13] Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(58) Exchange +(59) Exchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(59) Sort [codegen id : 20] +(60) Sort [codegen id : 14] Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST], false, 0 -(60) SortMergeJoin [codegen id : 21] +(61) SortMergeJoin [codegen id : 15] Left keys [2]: [ws_order_number#43, ws_item_sk#42] Right keys [2]: [wr_order_number#56, wr_item_sk#55] Join type: LeftOuter Join condition: None -(61) Project [codegen id : 21] +(62) Project [codegen id : 15] Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(62) Union +(63) Union -(63) HashAggregate [codegen id : 22] +(64) HashAggregate [codegen id : 16] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(64) Exchange +(65) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(65) HashAggregate [codegen id : 23] +(66) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(66) HashAggregate [codegen id : 23] +(67) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum#62, sum#63] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -(67) Exchange +(68) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(68) HashAggregate [codegen id : 24] +(69) HashAggregate [codegen id : 18] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] -(69) Filter [codegen id : 24] +(70) Filter [codegen id : 18] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] Condition : isnotnull(sales_cnt#68) -(70) Exchange +(71) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(71) Sort [codegen id : 25] +(72) Sort [codegen id : 19] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 -(72) Scan parquet spark_catalog.default.catalog_sales +(73) Scan parquet spark_catalog.default.catalog_sales Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] @@ -462,65 +468,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(73) CometFilter +(74) CometFilter Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Condition : isnotnull(cs_item_sk#70) -(74) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] - -(75) ReusedExchange [Reuses operator id: 8] +(75) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(76) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#70] -Right keys [1]: [i_item_sk#76] -Join type: Inner -Join condition: None +(76) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Right output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70], [i_item_sk#76], Inner, BuildRight -(77) Project [codegen id : 28] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(77) CometProject Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(78) ReusedExchange [Reuses operator id: 137] +(78) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#81, d_year#82] +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 -(79) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#74] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None +(79) CometFilter +Input [2]: [d_date_sk#81, d_year#82] +Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(80) Project [codegen id : 28] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +(80) CometBroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] +Arguments: [d_date_sk#81, d_year#82] + +(81) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Right output [2]: [d_date_sk#81, d_year#82] +Arguments: [cs_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight + +(82) CometProject Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] + +(83) ColumnarToRow [codegen id : 20] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -(81) Exchange +(84) Exchange Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(82) Sort [codegen id : 29] +(85) Sort [codegen id : 21] Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 -(83) ReusedExchange [Reuses operator id: 20] +(86) ReusedExchange [Reuses operator id: 21] Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(84) Sort [codegen id : 31] +(87) Sort [codegen id : 23] Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] Arguments: [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST], false, 0 -(85) SortMergeJoin [codegen id : 32] +(88) SortMergeJoin [codegen id : 24] Left keys [2]: [cs_order_number#71, cs_item_sk#70] Right keys [2]: [cr_order_number#84, cr_item_sk#83] Join type: LeftOuter Join condition: None -(86) Project [codegen id : 32] +(89) Project [codegen id : 24] Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#20, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#21] Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(87) Scan parquet spark_catalog.default.store_sales +(90) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] @@ -528,65 +544,63 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(88) CometFilter +(91) CometFilter Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Condition : isnotnull(ss_item_sk#87) -(89) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] - -(90) ReusedExchange [Reuses operator id: 8] +(92) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(91) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#87] -Right keys [1]: [i_item_sk#93] -Join type: Inner -Join condition: None +(93) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Right output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +Arguments: [ss_item_sk#87], [i_item_sk#93], Inner, BuildRight -(92) Project [codegen id : 35] -Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(94) CometProject Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(93) ReusedExchange [Reuses operator id: 137] +(95) ReusedExchange [Reuses operator id: 80] Output [2]: [d_date_sk#98, d_year#99] -(94) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#91] -Right keys [1]: [d_date_sk#98] -Join type: Inner -Join condition: None +(96) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +Right output [2]: [d_date_sk#98, d_year#99] +Arguments: [ss_sold_date_sk#91], [d_date_sk#98], Inner, BuildRight -(95) Project [codegen id : 35] -Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +(97) CometProject Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] +Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -(96) Exchange +(98) ColumnarToRow [codegen id : 25] Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(97) Sort [codegen id : 36] +(99) Exchange +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(100) Sort [codegen id : 26] Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 -(98) ReusedExchange [Reuses operator id: 39] +(101) ReusedExchange [Reuses operator id: 40] Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -(99) Sort [codegen id : 38] +(102) Sort [codegen id : 28] Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] Arguments: [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST], false, 0 -(100) SortMergeJoin [codegen id : 39] +(103) SortMergeJoin [codegen id : 29] Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] Right keys [2]: [sr_ticket_number#101, sr_item_sk#100] Join type: LeftOuter Join condition: None -(101) Project [codegen id : 39] +(104) Project [codegen id : 29] Output [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#40, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#41] Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -(102) Scan parquet spark_catalog.default.web_sales +(105) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] @@ -594,186 +608,184 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(103) CometFilter +(106) CometFilter Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Condition : isnotnull(ws_item_sk#104) -(104) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] - -(105) ReusedExchange [Reuses operator id: 8] +(107) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(106) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#104] -Right keys [1]: [i_item_sk#110] -Join type: Inner -Join condition: None +(108) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104], [i_item_sk#110], Inner, BuildRight -(107) Project [codegen id : 42] -Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +(109) CometProject Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(108) ReusedExchange [Reuses operator id: 137] +(110) ReusedExchange [Reuses operator id: 80] Output [2]: [d_date_sk#115, d_year#116] -(109) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#108] -Right keys [1]: [d_date_sk#115] -Join type: Inner -Join condition: None +(111) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Right output [2]: [d_date_sk#115, d_year#116] +Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner, BuildRight -(110) Project [codegen id : 42] -Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +(112) CometProject Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] + +(113) ColumnarToRow [codegen id : 30] +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -(111) Exchange +(114) Exchange Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(112) Sort [codegen id : 43] +(115) Sort [codegen id : 31] Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] Arguments: [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST], false, 0 -(113) ReusedExchange [Reuses operator id: 58] +(116) ReusedExchange [Reuses operator id: 59] Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(114) Sort [codegen id : 45] +(117) Sort [codegen id : 33] Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] Arguments: [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST], false, 0 -(115) SortMergeJoin [codegen id : 46] +(118) SortMergeJoin [codegen id : 34] Left keys [2]: [ws_order_number#105, ws_item_sk#104] Right keys [2]: [wr_order_number#118, wr_item_sk#117] Join type: LeftOuter Join condition: None -(116) Project [codegen id : 46] +(119) Project [codegen id : 34] Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#60, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#61] Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(117) Union +(120) Union -(118) HashAggregate [codegen id : 47] +(121) HashAggregate [codegen id : 35] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(119) Exchange +(122) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(120) HashAggregate [codegen id : 48] +(123) HashAggregate [codegen id : 36] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(121) HashAggregate [codegen id : 48] +(124) HashAggregate [codegen id : 36] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum#62, sum#121] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -(122) Exchange +(125) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(123) HashAggregate [codegen id : 49] +(126) HashAggregate [codegen id : 37] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#20)#66 AS sales_cnt#123, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#124] -(124) Filter [codegen id : 49] +(127) Filter [codegen id : 37] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] Condition : isnotnull(sales_cnt#123) -(125) Exchange +(128) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(126) Sort [codegen id : 50] +(129) Sort [codegen id : 38] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 -(127) SortMergeJoin [codegen id : 51] +(130) SortMergeJoin [codegen id : 39] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Join type: Inner Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) -(128) Project [codegen id : 51] +(131) Project [codegen id : 39] Output [10]: [d_year#82 AS prev_year#125, d_year#14 AS year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#123 AS prev_yr_cnt#127, sales_cnt#68 AS curr_yr_cnt#128, (sales_cnt#68 - sales_cnt#123) AS sales_cnt_diff#129, (sales_amt#69 - sales_amt#124) AS sales_amt_diff#130] Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -(129) TakeOrderedAndProject +(132) TakeOrderedAndProject Input [10]: [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] Arguments: 100, [sales_cnt_diff#129 ASC NULLS FIRST], [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (133) -+- * ColumnarToRow (132) - +- CometFilter (131) - +- CometScan parquet spark_catalog.default.date_dim (130) +BroadcastExchange (136) ++- * ColumnarToRow (135) + +- CometFilter (134) + +- CometScan parquet spark_catalog.default.date_dim (133) -(130) Scan parquet spark_catalog.default.date_dim +(133) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(131) CometFilter +(134) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(132) ColumnarToRow [codegen id : 1] +(135) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(133) BroadcastExchange +(136) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 -BroadcastExchange (137) -+- * ColumnarToRow (136) - +- CometFilter (135) - +- CometScan parquet spark_catalog.default.date_dim (134) +Subquery:4 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (140) ++- * ColumnarToRow (139) + +- CometFilter (138) + +- CometScan parquet spark_catalog.default.date_dim (137) -(134) Scan parquet spark_catalog.default.date_dim +(137) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#81, d_year#82] 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 -(135) CometFilter +(138) CometFilter Input [2]: [d_date_sk#81, d_year#82] Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(136) ColumnarToRow [codegen id : 1] +(139) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] -(137) BroadcastExchange +(140) BroadcastExchange Input [2]: [d_date_sk#81, d_year#82] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt index 44bcabcdb1..34866bc37e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt @@ -1,41 +1,41 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_amt_diff] - WholeStageCodegen (51) + WholeStageCodegen (39) Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter - WholeStageCodegen (25) + WholeStageCodegen (19) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (24) + WholeStageCodegen (18) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - WholeStageCodegen (23) + WholeStageCodegen (17) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - WholeStageCodegen (22) + WholeStageCodegen (16) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (7) + WholeStageCodegen (5) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (2) Sort [cs_order_number,cs_item_sk] InputAdapter Exchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -45,196 +45,185 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometBroadcastExchange #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometBroadcastExchange #7 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (4) Sort [cr_order_number,cr_item_sk] InputAdapter - Exchange [cr_order_number,cr_item_sk] #7 - WholeStageCodegen (5) + Exchange [cr_order_number,cr_item_sk] #8 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - WholeStageCodegen (14) + WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (7) Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #9 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (9) Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #9 - WholeStageCodegen (12) + Exchange [sr_ticket_number,sr_item_sk] #10 + WholeStageCodegen (8) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - WholeStageCodegen (21) + WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (18) + WholeStageCodegen (12) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (17) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ws_order_number,ws_item_sk] #11 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter - WholeStageCodegen (20) + WholeStageCodegen (14) Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_order_number,wr_item_sk] #11 - WholeStageCodegen (19) + Exchange [wr_order_number,wr_item_sk] #12 + WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter - WholeStageCodegen (50) + WholeStageCodegen (38) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - WholeStageCodegen (49) + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + WholeStageCodegen (37) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (48) + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + WholeStageCodegen (36) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - WholeStageCodegen (47) + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + WholeStageCodegen (35) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (32) + WholeStageCodegen (24) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (29) + WholeStageCodegen (21) Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (28) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [cs_order_number,cs_item_sk] #16 + WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 + BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange #18 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (31) + WholeStageCodegen (23) Sort [cr_order_number,cr_item_sk] InputAdapter - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - WholeStageCodegen (39) + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + WholeStageCodegen (29) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (36) + WholeStageCodegen (26) Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (35) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #19 + WholeStageCodegen (25) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter - WholeStageCodegen (38) + WholeStageCodegen (28) Sort [sr_ticket_number,sr_item_sk] InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - WholeStageCodegen (46) + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + WholeStageCodegen (34) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (43) + WholeStageCodegen (31) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (42) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ws_order_number,ws_item_sk] #20 + WholeStageCodegen (30) + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter - WholeStageCodegen (45) + WholeStageCodegen (33) Sort [wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index f3b80d8bdf..27a4a8c3f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -1,42 +1,38 @@ == Physical Plan == -TakeOrderedAndProject (38) -+- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- Union (34) - :- * Project (15) - : +- * BroadcastHashJoin Inner BuildRight (14) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (13) - : +- * ColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan parquet spark_catalog.default.date_dim (10) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.web_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * ColumnarToRow (27) - : : +- CometFilter (26) - : : +- CometScan parquet spark_catalog.default.catalog_sales (25) - : +- ReusedExchange (28) - +- ReusedExchange (31) +TakeOrderedAndProject (34) ++- * HashAggregate (33) + +- Exchange (32) + +- * ColumnarToRow (31) + +- CometHashAggregate (30) + +- CometUnion (29) + :- CometProject (12) + : +- CometBroadcastHashJoin (11) + : :- CometProject (7) + : : +- CometBroadcastHashJoin (6) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.item (3) + : +- CometBroadcastExchange (10) + : +- CometFilter (9) + : +- CometScan parquet spark_catalog.default.date_dim (8) + :- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.web_sales (13) + : : +- ReusedExchange (15) + : +- ReusedExchange (18) + +- CometProject (28) + +- CometBroadcastHashJoin (27) + :- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.catalog_sales (21) + : +- ReusedExchange (23) + +- ReusedExchange (26) (1) Scan parquet spark_catalog.default.store_sales @@ -51,66 +47,55 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [i_item_sk#5, i_category#6] Condition : isnotnull(i_item_sk#5) -(6) ColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_category#6] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [2]: [i_item_sk#5, i_category#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#5, i_category#6] -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Right output [2]: [i_item_sk#5, i_category#6] +Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) Project [codegen id : 3] -Output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6] +(7) CometProject Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#6] +Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6], [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6] -(10) Scan parquet spark_catalog.default.date_dim +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Condition : isnotnull(d_date_sk#7) -(12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [d_date_sk#7, d_year#8, d_qoy#9] -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6] +Right output [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Arguments: [ss_sold_date_sk#4], [d_date_sk#7], Inner, BuildRight -(15) Project [codegen id : 3] -Output [6]: [store AS channel#10, ss_store_sk#2 AS col_name#11, d_year#8, d_qoy#9, i_category#6, ss_ext_sales_price#3 AS ext_sales_price#12] +(12) CometProject Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6, d_date_sk#7, d_year#8, d_qoy#9] +Arguments: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12], [store AS channel#10, ss_store_sk#2 AS col_name#11, d_year#8, d_qoy#9, i_category#6, ss_ext_sales_price#3 AS ext_sales_price#12] -(16) Scan parquet spark_catalog.default.web_sales +(13) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] @@ -118,40 +103,35 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#16)] PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] Condition : (isnull(ws_ship_customer_sk#14) AND isnotnull(ws_item_sk#13)) -(18) ColumnarToRow [codegen id : 6] -Input [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] - -(19) ReusedExchange [Reuses operator id: 7] +(15) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#17, i_category#18] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#13] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] +Right output [2]: [i_item_sk#17, i_category#18] +Arguments: [ws_item_sk#13], [i_item_sk#17], Inner, BuildRight -(21) Project [codegen id : 6] -Output [4]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18] +(17) CometProject Input [6]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_item_sk#17, i_category#18] +Arguments: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18], [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18] -(22) ReusedExchange [Reuses operator id: 13] +(18) ReusedExchange [Reuses operator id: 10] Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None +(19) CometBroadcastHashJoin +Left output [4]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18] +Right output [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Arguments: [ws_sold_date_sk#16], [d_date_sk#19], Inner, BuildRight -(24) Project [codegen id : 6] -Output [6]: [web AS channel#22, ws_ship_customer_sk#14 AS col_name#23, d_year#20, d_qoy#21, i_category#18, ws_ext_sales_price#15 AS ext_sales_price#24] +(20) CometProject Input [7]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18, d_date_sk#19, d_year#20, d_qoy#21] +Arguments: [channel#22, col_name#23, d_year#20, d_qoy#21, i_category#18, ext_sales_price#24], [web AS channel#22, ws_ship_customer_sk#14 AS col_name#23, d_year#20, d_qoy#21, i_category#18, ws_ext_sales_price#15 AS ext_sales_price#24] -(25) Scan parquet spark_catalog.default.catalog_sales +(21) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] @@ -159,60 +139,59 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#28)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(26) CometFilter +(22) CometFilter Input [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] Condition : (isnull(cs_ship_addr_sk#25) AND isnotnull(cs_item_sk#26)) -(27) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] - -(28) ReusedExchange [Reuses operator id: 7] +(23) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#29, i_category#30] -(29) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#26] -Right keys [1]: [i_item_sk#29] -Join type: Inner -Join condition: None +(24) CometBroadcastHashJoin +Left output [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] +Right output [2]: [i_item_sk#29, i_category#30] +Arguments: [cs_item_sk#26], [i_item_sk#29], Inner, BuildRight -(30) Project [codegen id : 9] -Output [4]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30] +(25) CometProject Input [6]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28, i_item_sk#29, i_category#30] +Arguments: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30], [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30] -(31) ReusedExchange [Reuses operator id: 13] +(26) ReusedExchange [Reuses operator id: 10] Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#28] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [4]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30] +Right output [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Arguments: [cs_sold_date_sk#28], [d_date_sk#31], Inner, BuildRight -(33) Project [codegen id : 9] -Output [6]: [catalog AS channel#34, cs_ship_addr_sk#25 AS col_name#35, d_year#32, d_qoy#33, i_category#30, cs_ext_sales_price#27 AS ext_sales_price#36] +(28) CometProject Input [7]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30, d_date_sk#31, d_year#32, d_qoy#33] +Arguments: [channel#34, col_name#35, d_year#32, d_qoy#33, i_category#30, ext_sales_price#36], [catalog AS channel#34, cs_ship_addr_sk#25 AS col_name#35, d_year#32, d_qoy#33, i_category#30, cs_ext_sales_price#27 AS ext_sales_price#36] -(34) Union +(29) CometUnion +Child 0 Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12] +Child 1 Input [6]: [channel#22, col_name#23, d_year#20, d_qoy#21, i_category#18, ext_sales_price#24] +Child 2 Input [6]: [channel#34, col_name#35, d_year#32, d_qoy#33, i_category#30, ext_sales_price#36] -(35) HashAggregate [codegen id : 10] +(30) CometHashAggregate Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12] Keys [5]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#12))] -Aggregate Attributes [2]: [count#37, sum#38] -Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] -(36) Exchange -Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] -Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(31) ColumnarToRow [codegen id : 1] +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#37, sum#38] + +(32) Exchange +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#37, sum#38] +Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(37) HashAggregate [codegen id : 11] -Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] +(33) HashAggregate [codegen id : 2] +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#37, sum#38] Keys [5]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] -Aggregate Attributes [2]: [count(1)#41, sum(UnscaledValue(ext_sales_price#12))#42] -Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count(1)#41 AS sales_cnt#43, MakeDecimal(sum(UnscaledValue(ext_sales_price#12))#42,17,2) AS sales_amt#44] +Aggregate Attributes [2]: [count(1)#39, sum(UnscaledValue(ext_sales_price#12))#40] +Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count(1)#39 AS sales_cnt#41, MakeDecimal(sum(UnscaledValue(ext_sales_price#12))#40,17,2) AS sales_amt#42] -(38) TakeOrderedAndProject -Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] -Arguments: 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] +(34) TakeOrderedAndProject +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#41, sales_amt#42] +Arguments: 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#41, sales_amt#42] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt index 73e6b09afe..473eef9e46 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt @@ -1,58 +1,38 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - WholeStageCodegen (11) + WholeStageCodegen (2) HashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] [count(1),sum(UnscaledValue(ext_sales_price)),sales_cnt,sales_amt,count,sum] InputAdapter Exchange [channel,col_name,d_year,d_qoy,i_category] #1 - WholeStageCodegen (10) - HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Union - WholeStageCodegen (3) - Project [ss_store_sk,d_year,d_qoy,i_category,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ss_store_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - WholeStageCodegen (6) - Project [ws_ship_customer_sk,d_year,d_qoy,i_category,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [ws_ship_customer_sk,ws_item_sk] - CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] - InputAdapter - ReusedExchange [i_item_sk,i_category] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - WholeStageCodegen (9) - Project [cs_ship_addr_sk,d_year,d_qoy,i_category,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_ship_addr_sk,cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - InputAdapter - ReusedExchange [i_item_sk,i_category] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 + CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometUnion + CometProject [ss_store_sk,ss_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange #2 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometBroadcastExchange #3 + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometFilter [ws_ship_customer_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_ship_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index bbfa6a4c4a..a61e0be7dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -1,89 +1,90 @@ == Physical Plan == -TakeOrderedAndProject (85) -+- * HashAggregate (84) - +- Exchange (83) - +- * HashAggregate (82) - +- * Expand (81) - +- Union (80) - :- * Project (30) - : +- * BroadcastHashJoin LeftOuter BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.store_returns (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - :- * Project (49) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (48) - : :- BroadcastExchange (39) - : : +- * HashAggregate (38) - : : +- Exchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * ColumnarToRow (32) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : +- ReusedExchange (33) - : +- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * ColumnarToRow (41) - : : +- CometScan parquet spark_catalog.default.catalog_returns (40) - : +- ReusedExchange (42) - +- * Project (79) - +- * BroadcastHashJoin LeftOuter BuildRight (78) - :- * HashAggregate (64) - : +- Exchange (63) - : +- * HashAggregate (62) - : +- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Project (55) - : : +- * BroadcastHashJoin Inner BuildRight (54) - : : :- * ColumnarToRow (52) - : : : +- CometFilter (51) - : : : +- CometScan parquet spark_catalog.default.web_sales (50) - : : +- ReusedExchange (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_page (56) - +- BroadcastExchange (77) - +- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * ColumnarToRow (67) - : : +- CometFilter (66) - : : +- CometScan parquet spark_catalog.default.web_returns (65) - : +- ReusedExchange (68) - +- ReusedExchange (71) +TakeOrderedAndProject (86) ++- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Expand (82) + +- Union (81) + :- * Project (32) + : +- * BroadcastHashJoin LeftOuter BuildRight (31) + : :- * HashAggregate (17) + : : +- Exchange (16) + : : +- * ColumnarToRow (15) + : : +- CometHashAggregate (14) + : : +- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- BroadcastExchange (30) + : +- * HashAggregate (29) + : +- Exchange (28) + : +- * ColumnarToRow (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.store_returns (18) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + :- * Project (51) + : +- * BroadcastNestedLoopJoin Inner BuildLeft (50) + : :- BroadcastExchange (41) + : : +- * HashAggregate (40) + : : +- Exchange (39) + : : +- * ColumnarToRow (38) + : : +- CometHashAggregate (37) + : : +- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometScan parquet spark_catalog.default.catalog_sales (33) + : : +- ReusedExchange (34) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * ColumnarToRow (47) + : +- CometHashAggregate (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometScan parquet spark_catalog.default.catalog_returns (42) + : +- ReusedExchange (43) + +- * Project (80) + +- * BroadcastHashJoin LeftOuter BuildRight (79) + :- * HashAggregate (65) + : +- Exchange (64) + : +- * ColumnarToRow (63) + : +- CometHashAggregate (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (56) + : : +- CometBroadcastHashJoin (55) + : : :- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.web_sales (52) + : : +- ReusedExchange (54) + : +- CometBroadcastExchange (59) + : +- CometFilter (58) + : +- CometScan parquet spark_catalog.default.web_page (57) + +- BroadcastExchange (78) + +- * HashAggregate (77) + +- Exchange (76) + +- * ColumnarToRow (75) + +- CometHashAggregate (74) + +- CometProject (73) + +- CometBroadcastHashJoin (72) + :- CometProject (70) + : +- CometBroadcastHashJoin (69) + : :- CometFilter (67) + : : +- CometScan parquet spark_catalog.default.web_returns (66) + : +- ReusedExchange (68) + +- ReusedExchange (71) (1) Scan parquet spark_catalog.default.store_sales @@ -98,450 +99,438 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_date#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] + +(7) CometBroadcastHashJoin +Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +(8) CometProject Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -(7) Scan parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] +(9) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) +(10) CometFilter +Input [1]: [s_store_sk#8] +Condition : isnotnull(s_store_sk#8) -(9) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] +(11) CometBroadcastExchange +Input [1]: [s_store_sk#8] +Arguments: [s_store_sk#8] -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +Right output [1]: [s_store_sk#8] +Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +(13) CometProject +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] +(14) CometHashAggregate +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Keys [1]: [s_store_sk#8] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] -(14) Exchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(15) ColumnarToRow [codegen id : 1] +Input [3]: [s_store_sk#8, sum#9, sum#10] -(15) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] +(16) Exchange +Input [3]: [s_store_sk#8, sum#9, sum#10] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(17) HashAggregate [codegen id : 4] +Input [3]: [s_store_sk#8, sum#9, sum#10] +Keys [1]: [s_store_sk#8] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#11, sum(UnscaledValue(ss_net_profit#3))#12] +Results [3]: [s_store_sk#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS sales#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#12,17,2) AS profit#14] -(16) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +(18) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#19)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(17) CometFilter -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) +(19) CometFilter +Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] +Condition : isnotnull(sr_store_sk#15) -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#20] -(19) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#21] +(21) CometBroadcastHashJoin +Left output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] +Right output [1]: [d_date_sk#20] +Arguments: [sr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None +(22) CometProject +Input [5]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18, d_date_sk#20] +Arguments: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17], [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] -(21) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] +(23) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#21] -(22) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#22] +(24) CometBroadcastHashJoin +Left output [3]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] +Right output [1]: [s_store_sk#21] +Arguments: [sr_store_sk#15], [s_store_sk#21], Inner, BuildRight -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None +(25) CometProject +Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, s_store_sk#21] +Arguments: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21], [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] + +(26) CometHashAggregate +Input [3]: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] +Keys [1]: [s_store_sk#21] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#16)), partial_sum(UnscaledValue(sr_net_loss#17))] + +(27) ColumnarToRow [codegen id : 2] +Input [3]: [s_store_sk#21, sum#22, sum#23] -(24) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] - -(25) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Keys [1]: [s_store_sk#22] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#23, sum#24] -Results [3]: [s_store_sk#22, sum#25, sum#26] - -(26) Exchange -Input [3]: [s_store_sk#22, sum#25, sum#26] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#22, sum#25, sum#26] -Keys [1]: [s_store_sk#22] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] -Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] - -(28) BroadcastExchange -Input [3]: [s_store_sk#22, returns#29, profit_loss#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#22] +(28) Exchange +Input [3]: [s_store_sk#21, sum#22, sum#23] +Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(29) HashAggregate [codegen id : 3] +Input [3]: [s_store_sk#21, sum#22, sum#23] +Keys [1]: [s_store_sk#21] +Functions [2]: [sum(UnscaledValue(sr_return_amt#16)), sum(UnscaledValue(sr_net_loss#17))] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#16))#24, sum(UnscaledValue(sr_net_loss#17))#25] +Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#24,17,2) AS returns#26, MakeDecimal(sum(UnscaledValue(sr_net_loss#17))#25,17,2) AS profit_loss#27] + +(30) BroadcastExchange +Input [3]: [s_store_sk#21, returns#26, profit_loss#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(31) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [s_store_sk#8] +Right keys [1]: [s_store_sk#21] Join type: LeftOuter Join condition: None -(30) Project [codegen id : 8] -Output [5]: [sales#14, coalesce(returns#29, 0.00) AS returns#31, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#32, store channel AS channel#33, s_store_sk#7 AS id#34] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] +(32) Project [codegen id : 4] +Output [5]: [sales#13, coalesce(returns#26, 0.00) AS returns#28, (profit#14 - coalesce(profit_loss#27, 0.00)) AS profit#29, store channel AS channel#30, s_store_sk#8 AS id#31] +Input [6]: [s_store_sk#8, sales#13, profit#14, s_store_sk#21, returns#26, profit_loss#27] -(31) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(33) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(cs_sold_date_sk#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(34) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#37] -(33) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#40] +(35) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] +Right output [1]: [d_date_sk#37] +Arguments: [cs_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None +(36) CometProject +Input [5]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35, d_date_sk#37] +Arguments: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34], [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] + +(37) CometHashAggregate +Input [3]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] +Keys [1]: [cs_call_center_sk#32] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#33)), partial_sum(UnscaledValue(cs_net_profit#34))] -(35) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] - -(36) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum#41, sum#42] -Results [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(37) Exchange -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(38) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] -Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] - -(39) BroadcastExchange -Input [3]: [cs_call_center_sk#35, sales#47, profit#48] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(40) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(38) ColumnarToRow [codegen id : 5] +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] + +(39) Exchange +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +Arguments: hashpartitioning(cs_call_center_sk#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(40) HashAggregate [codegen id : 6] +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +Keys [1]: [cs_call_center_sk#32] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#33)), sum(UnscaledValue(cs_net_profit#34))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#33))#40, sum(UnscaledValue(cs_net_profit#34))#41] +Results [3]: [cs_call_center_sk#32, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#33))#40,17,2) AS sales#42, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#41,17,2) AS profit#43] + +(41) BroadcastExchange +Input [3]: [cs_call_center_sk#32, sales#42, profit#43] +Arguments: IdentityBroadcastMode, [plan_id=5] + +(42) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(cr_returned_date_sk#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(43) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#48] -(42) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#53] - -(43) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#51] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +Right output [1]: [d_date_sk#48] +Arguments: [cr_returned_date_sk#46], [d_date_sk#48], Inner, BuildRight -(44) Project [codegen id : 13] -Output [2]: [cr_return_amount#49, cr_net_loss#50] -Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] +(45) CometProject +Input [4]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46, d_date_sk#48] +Arguments: [cr_return_amount#44, cr_net_loss#45], [cr_return_amount#44, cr_net_loss#45] -(45) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#49, cr_net_loss#50] +(46) CometHashAggregate +Input [2]: [cr_return_amount#44, cr_net_loss#45] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum#54, sum#55] -Results [2]: [sum#56, sum#57] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#44)), partial_sum(UnscaledValue(cr_net_loss#45))] -(46) Exchange -Input [2]: [sum#56, sum#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(47) ColumnarToRow [codegen id : 7] +Input [2]: [sum#49, sum#50] -(47) HashAggregate -Input [2]: [sum#56, sum#57] +(48) Exchange +Input [2]: [sum#49, sum#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] + +(49) HashAggregate +Input [2]: [sum#49, sum#50] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] +Functions [2]: [sum(UnscaledValue(cr_return_amount#44)), sum(UnscaledValue(cr_net_loss#45))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#44))#51, sum(UnscaledValue(cr_net_loss#45))#52] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#44))#51,17,2) AS returns#53, MakeDecimal(sum(UnscaledValue(cr_net_loss#45))#52,17,2) AS profit_loss#54] -(48) BroadcastNestedLoopJoin [codegen id : 14] +(50) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(49) Project [codegen id : 14] -Output [5]: [sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#62, catalog channel AS channel#63, cs_call_center_sk#35 AS id#64] -Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] +(51) Project [codegen id : 8] +Output [5]: [sales#42, returns#53, (profit#43 - profit_loss#54) AS profit#55, catalog channel AS channel#56, cs_call_center_sk#32 AS id#57] +Input [5]: [cs_call_center_sk#32, sales#42, profit#43, returns#53, profit_loss#54] -(50) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(52) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(51) CometFilter -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_web_page_sk#65) - -(52) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(53) CometFilter +Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] +Condition : isnotnull(ws_web_page_sk#58) -(53) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#70] +(54) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#63] -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#68] -Right keys [1]: [d_date_sk#70] -Join type: Inner -Join condition: None +(55) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] +Right output [1]: [d_date_sk#63] +Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner, BuildRight -(55) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] -Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] +(56) CometProject +Input [5]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61, d_date_sk#63] +Arguments: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60], [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] -(56) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#71] +(57) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(57) CometFilter -Input [1]: [wp_web_page_sk#71] -Condition : isnotnull(wp_web_page_sk#71) +(58) CometFilter +Input [1]: [wp_web_page_sk#64] +Condition : isnotnull(wp_web_page_sk#64) -(58) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#71] +(59) CometBroadcastExchange +Input [1]: [wp_web_page_sk#64] +Arguments: [wp_web_page_sk#64] -(59) BroadcastExchange -Input [1]: [wp_web_page_sk#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(60) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] +Right output [1]: [wp_web_page_sk#64] +Arguments: [ws_web_page_sk#58], [wp_web_page_sk#64], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#65] -Right keys [1]: [wp_web_page_sk#71] -Join type: Inner -Join condition: None +(61) CometProject +Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +Arguments: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64], [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] + +(62) CometHashAggregate +Input [3]: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +Keys [1]: [wp_web_page_sk#64] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#59)), partial_sum(UnscaledValue(ws_net_profit#60))] -(61) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] - -(62) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum#72, sum#73] -Results [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(63) Exchange -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(64) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] -Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] - -(65) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(63) ColumnarToRow [codegen id : 9] +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] + +(64) Exchange +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +Arguments: hashpartitioning(wp_web_page_sk#64, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(65) HashAggregate [codegen id : 12] +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +Keys [1]: [wp_web_page_sk#64] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#59)), sum(UnscaledValue(ws_net_profit#60))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#59))#67, sum(UnscaledValue(ws_net_profit#60))#68] +Results [3]: [wp_web_page_sk#64, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#67,17,2) AS sales#69, MakeDecimal(sum(UnscaledValue(ws_net_profit#60))#68,17,2) AS profit#70] + +(66) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(wr_returned_date_sk#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(66) CometFilter -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] -Condition : isnotnull(wr_web_page_sk#80) +(67) CometFilter +Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +Condition : isnotnull(wr_web_page_sk#71) -(67) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(68) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#76] -(68) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#85] +(69) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +Right output [1]: [d_date_sk#76] +Arguments: [wr_returned_date_sk#74], [d_date_sk#76], Inner, BuildRight -(69) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#83] -Right keys [1]: [d_date_sk#85] -Join type: Inner -Join condition: None - -(70) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] -Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] +(70) CometProject +Input [5]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74, d_date_sk#76] +Arguments: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73], [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] (71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#86] - -(72) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#80] -Right keys [1]: [wp_web_page_sk#86] -Join type: Inner -Join condition: None - -(73) Project [codegen id : 20] -Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] - -(74) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum#87, sum#88] -Results [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(75) Exchange -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(76) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] -Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] - -(77) BroadcastExchange -Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(78) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#71] -Right keys [1]: [wp_web_page_sk#86] +Output [1]: [wp_web_page_sk#77] + +(72) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] +Right output [1]: [wp_web_page_sk#77] +Arguments: [wr_web_page_sk#71], [wp_web_page_sk#77], Inner, BuildRight + +(73) CometProject +Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +Arguments: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77], [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] + +(74) CometHashAggregate +Input [3]: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#72)), partial_sum(UnscaledValue(wr_net_loss#73))] + +(75) ColumnarToRow [codegen id : 10] +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] + +(76) Exchange +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(77) HashAggregate [codegen id : 11] +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(wr_return_amt#72)), sum(UnscaledValue(wr_net_loss#73))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#72))#80, sum(UnscaledValue(wr_net_loss#73))#81] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(wr_return_amt#72))#80,17,2) AS returns#82, MakeDecimal(sum(UnscaledValue(wr_net_loss#73))#81,17,2) AS profit_loss#83] + +(78) BroadcastExchange +Input [3]: [wp_web_page_sk#77, returns#82, profit_loss#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +(79) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [wp_web_page_sk#64] +Right keys [1]: [wp_web_page_sk#77] Join type: LeftOuter Join condition: None -(79) Project [codegen id : 22] -Output [5]: [sales#78, coalesce(returns#93, 0.00) AS returns#95, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#96, web channel AS channel#97, wp_web_page_sk#71 AS id#98] -Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] +(80) Project [codegen id : 12] +Output [5]: [sales#69, coalesce(returns#82, 0.00) AS returns#84, (profit#70 - coalesce(profit_loss#83, 0.00)) AS profit#85, web channel AS channel#86, wp_web_page_sk#64 AS id#87] +Input [6]: [wp_web_page_sk#64, sales#69, profit#70, wp_web_page_sk#77, returns#82, profit_loss#83] -(80) Union +(81) Union -(81) Expand [codegen id : 23] -Input [5]: [sales#14, returns#31, profit#32, channel#33, id#34] -Arguments: [[sales#14, returns#31, profit#32, channel#33, id#34, 0], [sales#14, returns#31, profit#32, channel#33, null, 1], [sales#14, returns#31, profit#32, null, null, 3]], [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] +(82) Expand [codegen id : 13] +Input [5]: [sales#13, returns#28, profit#29, channel#30, id#31] +Arguments: [[sales#13, returns#28, profit#29, channel#30, id#31, 0], [sales#13, returns#28, profit#29, channel#30, null, 1], [sales#13, returns#28, profit#29, null, null, 3]], [sales#13, returns#28, profit#29, channel#88, id#89, spark_grouping_id#90] -(82) HashAggregate [codegen id : 23] -Input [6]: [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] -Keys [3]: [channel#99, id#100, spark_grouping_id#101] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#31), partial_sum(profit#32)] -Aggregate Attributes [6]: [sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] -Results [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +(83) HashAggregate [codegen id : 13] +Input [6]: [sales#13, returns#28, profit#29, channel#88, id#89, spark_grouping_id#90] +Keys [3]: [channel#88, id#89, spark_grouping_id#90] +Functions [3]: [partial_sum(sales#13), partial_sum(returns#28), partial_sum(profit#29)] +Aggregate Attributes [6]: [sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Results [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -(83) Exchange -Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Arguments: hashpartitioning(channel#99, id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(84) Exchange +Input [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Arguments: hashpartitioning(channel#88, id#89, spark_grouping_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(84) HashAggregate [codegen id : 24] -Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Keys [3]: [channel#99, id#100, spark_grouping_id#101] -Functions [3]: [sum(sales#14), sum(returns#31), sum(profit#32)] -Aggregate Attributes [3]: [sum(sales#14)#114, sum(returns#31)#115, sum(profit#32)#116] -Results [5]: [channel#99, id#100, sum(sales#14)#114 AS sales#117, sum(returns#31)#115 AS returns#118, sum(profit#32)#116 AS profit#119] +(85) HashAggregate [codegen id : 14] +Input [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Keys [3]: [channel#88, id#89, spark_grouping_id#90] +Functions [3]: [sum(sales#13), sum(returns#28), sum(profit#29)] +Aggregate Attributes [3]: [sum(sales#13)#103, sum(returns#28)#104, sum(profit#29)#105] +Results [5]: [channel#88, id#89, sum(sales#13)#103 AS sales#106, sum(returns#28)#104 AS returns#107, sum(profit#29)#105 AS profit#108] -(85) TakeOrderedAndProject -Input [5]: [channel#99, id#100, sales#117, returns#118, profit#119] -Arguments: 100, [channel#99 ASC NULLS FIRST, id#100 ASC NULLS FIRST], [channel#99, id#100, sales#117, returns#118, profit#119] +(86) TakeOrderedAndProject +Input [5]: [channel#88, id#89, sales#106, returns#107, profit#108] +Arguments: 100, [channel#88 ASC NULLS FIRST, id#89 ASC NULLS FIRST], [channel#88, id#89, sales#106, returns#107, profit#108] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (90) -+- * ColumnarToRow (89) - +- CometProject (88) - +- CometFilter (87) - +- CometScan parquet spark_catalog.default.date_dim (86) +BroadcastExchange (91) ++- * ColumnarToRow (90) + +- CometProject (89) + +- CometFilter (88) + +- CometScan parquet spark_catalog.default.date_dim (87) -(86) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#120] +(87) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_date#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] ReadSchema: struct -(87) CometFilter -Input [2]: [d_date_sk#6, d_date#120] -Condition : (((isnotnull(d_date#120) AND (d_date#120 >= 2000-08-03)) AND (d_date#120 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) +(88) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) -(88) CometProject -Input [2]: [d_date_sk#6, d_date#120] +(89) CometProject +Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(89) ColumnarToRow [codegen id : 1] +(90) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(90) BroadcastExchange +(91) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#35 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 40 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 65 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index d6693067f0..590c59fdc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt @@ -1,27 +1,27 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (24) + WholeStageCodegen (14) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (23) + WholeStageCodegen (13) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (8) + WholeStageCodegen (4) Project [sales,returns,profit,profit_loss,s_store_sk] BroadcastHashJoin [s_store_sk,s_store_sk] HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [s_store_sk] #2 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -32,112 +32,99 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #5 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (3) HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [s_store_sk] #6 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [s_store_sk] #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometProject [sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometProject [sr_store_sk,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [s_store_sk] #4 - WholeStageCodegen (14) + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (8) Project [sales,returns,profit,profit_loss,cs_call_center_sk] BroadcastNestedLoopJoin InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) + BroadcastExchange #8 + WholeStageCodegen (6) HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [cs_call_center_sk] #8 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [cs_call_center_sk] #9 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange #9 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange #10 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [cr_return_amount,cr_net_loss] + CometProject [cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (12) Project [sales,returns,profit,profit_loss,wp_web_page_sk] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #10 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [wp_web_page_sk] #11 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #12 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter - BroadcastExchange #12 - WholeStageCodegen (21) + BroadcastExchange #13 + WholeStageCodegen (11) HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #13 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [wp_web_page_sk] #14 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] CometFilter [wr_web_page_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [wp_web_page_sk] #11 + ReusedExchange [d_date_sk] #4 + ReusedExchange [wp_web_page_sk] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt index c89bad2201..3bad7fc445 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt @@ -1,34 +1,35 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * BroadcastHashJoin Inner BuildRight (28) - :- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (27) - +- * ColumnarToRow (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.customer (24) +TakeOrderedAndProject (31) ++- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * HashAggregate (24) + : +- Exchange (23) + : +- * ColumnarToRow (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.household_demographics (15) + +- BroadcastExchange (28) + +- * ColumnarToRow (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) (1) Scan parquet spark_catalog.default.store_sales @@ -43,166 +44,171 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#10] +(4) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] + +(7) CometBroadcastHashJoin +Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight -(6) Project [codegen id : 4] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +(8) CometProject Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -(7) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#11, s_number_employees#12, s_city#13] +(9) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#13, s_number_employees#14, s_city#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Condition : (((isnotnull(s_number_employees#12) AND (s_number_employees#12 >= 200)) AND (s_number_employees#12 <= 295)) AND isnotnull(s_store_sk#11)) - -(9) CometProject -Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Arguments: [s_store_sk#11, s_city#13], [s_store_sk#11, s_city#13] +(10) CometFilter +Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] +Condition : (((isnotnull(s_number_employees#14) AND (s_number_employees#14 >= 200)) AND (s_number_employees#14 <= 295)) AND isnotnull(s_store_sk#13)) -(10) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_city#13] +(11) CometProject +Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] +Arguments: [s_store_sk#13, s_city#15], [s_store_sk#13, s_city#15] -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_city#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [2]: [s_store_sk#13, s_city#15] +Arguments: [s_store_sk#13, s_city#15] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +Right output [2]: [s_store_sk#13, s_city#15] +Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight -(13) Project [codegen id : 4] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11, s_city#13] +(14) CometProject +Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13, s_city#15] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -(14) Scan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] +(15) Scan parquet spark_catalog.default.household_demographics +Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : (((hd_dep_count#15 = 6) OR (hd_vehicle_count#16 > 2)) AND isnotnull(hd_demo_sk#14)) +(16) CometFilter +Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] +Condition : (((hd_dep_count#17 = 6) OR (hd_vehicle_count#18 > 2)) AND isnotnull(hd_demo_sk#16)) -(16) CometProject -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Arguments: [hd_demo_sk#14], [hd_demo_sk#14] +(17) CometProject +Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] +Arguments: [hd_demo_sk#16], [hd_demo_sk#16] -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#14] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#16] +Arguments: [hd_demo_sk#16] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(19) CometBroadcastHashJoin +Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] +Right output [1]: [hd_demo_sk#16] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] -Join type: Inner -Join condition: None +(20) CometProject +Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15, hd_demo_sk#16] +Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -(20) Project [codegen id : 4] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13, hd_demo_sk#14] - -(21) HashAggregate [codegen id : 4] -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] +(21) CometHashAggregate +Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum#17, sum#18] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -(22) Exchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) ColumnarToRow [codegen id : 1] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] + +(23) Exchange +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(23) HashAggregate [codegen id : 6] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] +(24) HashAggregate [codegen id : 3] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#21, sum(UnscaledValue(ss_net_profit#7))#22] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#21,17,2) AS amt#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#22,17,2) AS profit#24] +Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#21,17,2) AS amt#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#22,17,2) AS profit#24] -(24) Scan parquet spark_catalog.default.customer +(25) Scan parquet spark_catalog.default.customer Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(25) CometFilter +(26) CometFilter Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] Condition : isnotnull(c_customer_sk#25) -(26) ColumnarToRow [codegen id : 5] +(27) ColumnarToRow [codegen id : 2] Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -(27) BroadcastExchange +(28) BroadcastExchange Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(28) BroadcastHashJoin [codegen id : 6] +(29) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(29) Project [codegen id : 6] -Output [7]: [c_last_name#27, c_first_name#26, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#26, c_last_name#27] +(30) Project [codegen id : 3] +Output [7]: [c_last_name#27, c_first_name#26, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#15] +Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#23, profit#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -(30) TakeOrderedAndProject -Input [7]: [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Arguments: 100, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24] +(31) TakeOrderedAndProject +Input [7]: [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#15] +Arguments: 100, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (35) -+- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.date_dim (31) +BroadcastExchange (36) ++- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(31) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#29, d_dow#30] +(32) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_dow#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [d_date_sk#10, d_year#29, d_dow#30] -Condition : (((isnotnull(d_dow#30) AND (d_dow#30 = 1)) AND d_year#29 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) +(33) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] +Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(33) CometProject -Input [3]: [d_date_sk#10, d_year#29, d_dow#30] +(34) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(34) ColumnarToRow [codegen id : 1] +(35) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(35) BroadcastExchange +(36) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt index 4c05c449c5..b68a9474ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] InputAdapter Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] + CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -25,27 +25,21 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, CometProject [d_date_sk] CometFilter [d_dow,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk,s_city] - CometFilter [s_number_employees,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_dow,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometBroadcastExchange #4 + CometProject [s_store_sk,s_city] + CometFilter [s_number_employees,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] + CometBroadcastExchange #5 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) + BroadcastExchange #6 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index 7d6d717c1b..07bfd877c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -1,47 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (37) - +- * HashAggregate (36) - +- Exchange (35) - +- * HashAggregate (34) - +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * ColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.customer_address (13) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * ColumnarToRow (19) - : +- CometFilter (18) - : +- CometScan parquet spark_catalog.default.customer_address (17) - +- BroadcastExchange (24) - +- * ColumnarToRow (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan parquet spark_catalog.default.customer (20) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * ColumnarToRow (14) + : +- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.store (9) + +- BroadcastExchange (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * BroadcastHashJoin LeftSemi BuildRight (34) + :- * ColumnarToRow (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.customer_address (15) + +- BroadcastExchange (33) + +- * Project (32) + +- * Filter (31) + +- * HashAggregate (30) + +- Exchange (29) + +- * ColumnarToRow (28) + +- CometHashAggregate (27) + +- CometProject (26) + +- CometBroadcastHashJoin (25) + :- CometFilter (20) + : +- CometScan parquet spark_catalog.default.customer_address (19) + +- CometBroadcastExchange (24) + +- CometProject (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.customer (21) (1) Scan parquet spark_catalog.default.store_sales @@ -56,233 +57,238 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(7) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_store_name#7, s_zip#8] +(9) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#8, s_store_name#9, s_zip#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(8) CometFilter -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Condition : (isnotnull(s_store_sk#6) AND isnotnull(s_zip#8)) +(10) CometFilter +Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] +Condition : (isnotnull(s_store_sk#8) AND isnotnull(s_zip#10)) -(9) ColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] +(11) CometBroadcastExchange +Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] +Arguments: [s_store_sk#8, s_store_name#9, s_zip#10] -(10) BroadcastExchange -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [2]: [ss_store_sk#1, ss_net_profit#2] +Right output [3]: [s_store_sk#8, s_store_name#9, s_zip#10] +Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None +(13) CometProject +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_store_name#9, s_zip#10] +Arguments: [ss_net_profit#2, s_store_name#9, s_zip#10], [ss_net_profit#2, s_store_name#9, s_zip#10] -(12) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#8] +(14) ColumnarToRow [codegen id : 5] +Input [3]: [ss_net_profit#2, s_store_name#9, s_zip#10] -(13) Scan parquet spark_catalog.default.customer_address -Output [1]: [ca_zip#9] +(15) Scan parquet spark_catalog.default.customer_address +Output [1]: [ca_zip#11] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] ReadSchema: struct -(14) CometFilter -Input [1]: [ca_zip#9] -Condition : (substr(ca_zip#9, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#9, 1, 5))) +(16) CometFilter +Input [1]: [ca_zip#11] +Condition : (substr(ca_zip#11, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#11, 1, 5))) -(15) CometProject -Input [1]: [ca_zip#9] -Arguments: [ca_zip#10], [substr(ca_zip#9, 1, 5) AS ca_zip#10] +(17) CometProject +Input [1]: [ca_zip#11] +Arguments: [ca_zip#12], [substr(ca_zip#11, 1, 5) AS ca_zip#12] -(16) ColumnarToRow [codegen id : 6] -Input [1]: [ca_zip#10] +(18) ColumnarToRow [codegen id : 3] +Input [1]: [ca_zip#12] -(17) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#11, ca_zip#12] +(19) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#13, ca_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(18) CometFilter -Input [2]: [ca_address_sk#11, ca_zip#12] -Condition : isnotnull(ca_address_sk#11) - -(19) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#11, ca_zip#12] +(20) CometFilter +Input [2]: [ca_address_sk#13, ca_zip#14] +Condition : isnotnull(ca_address_sk#13) -(20) Scan parquet spark_catalog.default.customer -Output [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] +(21) Scan parquet spark_catalog.default.customer +Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(21) CometFilter -Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] -Condition : ((isnotnull(c_preferred_cust_flag#14) AND (c_preferred_cust_flag#14 = Y)) AND isnotnull(c_current_addr_sk#13)) +(22) CometFilter +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) -(22) CometProject -Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] -Arguments: [c_current_addr_sk#13], [c_current_addr_sk#13] +(23) CometProject +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Arguments: [c_current_addr_sk#15], [c_current_addr_sk#15] -(23) ColumnarToRow [codegen id : 3] -Input [1]: [c_current_addr_sk#13] +(24) CometBroadcastExchange +Input [1]: [c_current_addr_sk#15] +Arguments: [c_current_addr_sk#15] -(24) BroadcastExchange -Input [1]: [c_current_addr_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(25) CometBroadcastHashJoin +Left output [2]: [ca_address_sk#13, ca_zip#14] +Right output [1]: [c_current_addr_sk#15] +Arguments: [ca_address_sk#13], [c_current_addr_sk#15], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ca_address_sk#11] -Right keys [1]: [c_current_addr_sk#13] -Join type: Inner -Join condition: None +(26) CometProject +Input [3]: [ca_address_sk#13, ca_zip#14, c_current_addr_sk#15] +Arguments: [ca_zip#14], [ca_zip#14] -(26) Project [codegen id : 4] -Output [1]: [ca_zip#12] -Input [3]: [ca_address_sk#11, ca_zip#12, c_current_addr_sk#13] - -(27) HashAggregate [codegen id : 4] -Input [1]: [ca_zip#12] -Keys [1]: [ca_zip#12] +(27) CometHashAggregate +Input [1]: [ca_zip#14] +Keys [1]: [ca_zip#14] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [2]: [ca_zip#12, count#16] -(28) Exchange -Input [2]: [ca_zip#12, count#16] -Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(28) ColumnarToRow [codegen id : 1] +Input [2]: [ca_zip#14, count#17] -(29) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#12, count#16] -Keys [1]: [ca_zip#12] +(29) Exchange +Input [2]: [ca_zip#14, count#17] +Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(30) HashAggregate [codegen id : 2] +Input [2]: [ca_zip#14, count#17] +Keys [1]: [ca_zip#14] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [2]: [substr(ca_zip#12, 1, 5) AS ca_zip#18, count(1)#17 AS cnt#19] +Aggregate Attributes [1]: [count(1)#18] +Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#19, count(1)#18 AS cnt#20] -(30) Filter [codegen id : 5] -Input [2]: [ca_zip#18, cnt#19] -Condition : (cnt#19 > 10) +(31) Filter [codegen id : 2] +Input [2]: [ca_zip#19, cnt#20] +Condition : (cnt#20 > 10) -(31) Project [codegen id : 5] -Output [1]: [ca_zip#18] -Input [2]: [ca_zip#18, cnt#19] +(32) Project [codegen id : 2] +Output [1]: [ca_zip#19] +Input [2]: [ca_zip#19, cnt#20] -(32) BroadcastExchange -Input [1]: [ca_zip#18] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [plan_id=4] +(33) BroadcastExchange +Input [1]: [ca_zip#19] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [plan_id=2] -(33) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [coalesce(ca_zip#10, ), isnull(ca_zip#10)] -Right keys [2]: [coalesce(ca_zip#18, ), isnull(ca_zip#18)] +(34) BroadcastHashJoin [codegen id : 3] +Left keys [2]: [coalesce(ca_zip#12, ), isnull(ca_zip#12)] +Right keys [2]: [coalesce(ca_zip#19, ), isnull(ca_zip#19)] Join type: LeftSemi Join condition: None -(34) HashAggregate [codegen id : 6] -Input [1]: [ca_zip#10] -Keys [1]: [ca_zip#10] +(35) HashAggregate [codegen id : 3] +Input [1]: [ca_zip#12] +Keys [1]: [ca_zip#12] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#10] +Results [1]: [ca_zip#12] -(35) Exchange -Input [1]: [ca_zip#10] -Arguments: hashpartitioning(ca_zip#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(36) Exchange +Input [1]: [ca_zip#12] +Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(36) HashAggregate [codegen id : 7] -Input [1]: [ca_zip#10] -Keys [1]: [ca_zip#10] +(37) HashAggregate [codegen id : 4] +Input [1]: [ca_zip#12] +Keys [1]: [ca_zip#12] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#10] +Results [1]: [ca_zip#12] -(37) BroadcastExchange -Input [1]: [ca_zip#10] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=6] +(38) BroadcastExchange +Input [1]: [ca_zip#12] +Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=4] -(38) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [substr(s_zip#8, 1, 2)] -Right keys [1]: [substr(ca_zip#10, 1, 2)] +(39) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [substr(s_zip#10, 1, 2)] +Right keys [1]: [substr(ca_zip#12, 1, 2)] Join type: Inner Join condition: None -(39) Project [codegen id : 8] -Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#10] +(40) Project [codegen id : 5] +Output [2]: [ss_net_profit#2, s_store_name#9] +Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#10, ca_zip#12] -(40) HashAggregate [codegen id : 8] -Input [2]: [ss_net_profit#2, s_store_name#7] -Keys [1]: [s_store_name#7] +(41) HashAggregate [codegen id : 5] +Input [2]: [ss_net_profit#2, s_store_name#9] +Keys [1]: [s_store_name#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#20] -Results [2]: [s_store_name#7, sum#21] +Aggregate Attributes [1]: [sum#21] +Results [2]: [s_store_name#9, sum#22] -(41) Exchange -Input [2]: [s_store_name#7, sum#21] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(42) Exchange +Input [2]: [s_store_name#9, sum#22] +Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(42) HashAggregate [codegen id : 9] -Input [2]: [s_store_name#7, sum#21] -Keys [1]: [s_store_name#7] +(43) HashAggregate [codegen id : 6] +Input [2]: [s_store_name#9, sum#22] +Keys [1]: [s_store_name#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS sum(ss_net_profit)#23] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] +Results [2]: [s_store_name#9, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS sum(ss_net_profit)#24] -(43) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#23] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#23] +(44) TakeOrderedAndProject +Input [2]: [s_store_name#9, sum(ss_net_profit)#24] +Arguments: 100, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#24] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(44) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#24, d_qoy#25] +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2)) AND (d_year#24 = 1998)) AND isnotnull(d_date_sk#5)) +(46) CometFilter +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) -(46) CometProject -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] +(47) CometProject +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(47) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(48) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt index 76fa276931..adf5554172 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (9) + WholeStageCodegen (6) HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] InputAdapter Exchange [s_store_name] #1 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [s_store_name,ss_net_profit] [sum,sum] Project [ss_net_profit,s_store_name] BroadcastHashJoin [s_zip,ca_zip] - Project [ss_net_profit,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_net_profit,s_store_name,s_zip] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -23,22 +23,20 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] CometProject [d_date_sk] CometFilter [d_qoy,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_zip] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #4 + CometFilter [s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) + BroadcastExchange #5 + WholeStageCodegen (4) HashAggregate [ca_zip] InputAdapter - Exchange [ca_zip] #5 - WholeStageCodegen (6) + Exchange [ca_zip] #6 + WholeStageCodegen (3) HashAggregate [ca_zip] BroadcastHashJoin [ca_zip,ca_zip] ColumnarToRow @@ -47,26 +45,22 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] CometFilter [ca_zip] CometScan parquet spark_catalog.default.customer_address [ca_zip] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (2) Project [ca_zip] Filter [cnt] HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] InputAdapter - Exchange [ca_zip] #7 - WholeStageCodegen (4) - HashAggregate [ca_zip] [count,count] - Project [ca_zip] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter + Exchange [ca_zip] #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_zip] + CometProject [ca_zip] + CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [c_current_addr_sk] - CometFilter [c_preferred_cust_flag,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] + CometBroadcastExchange #9 + CometProject [c_current_addr_sk] + CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index 4d07337e60..67c8af0831 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt @@ -1,52 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (33) - : : +- * Filter (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * HashAggregate (28) - : : +- Exchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : +- BroadcastExchange (39) - : +- * ColumnarToRow (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.customer (36) - +- BroadcastExchange (45) - +- * ColumnarToRow (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.customer_address (42) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (43) + : +- * BroadcastHashJoin Inner BuildRight (42) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- Exchange (16) + : : : +- * ColumnarToRow (15) + : : : +- CometHashAggregate (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.customer_address (9) + : : +- BroadcastExchange (35) + : : +- * Filter (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * HashAggregate (31) + : : +- * HashAggregate (30) + : : +- Exchange (29) + : : +- * ColumnarToRow (28) + : : +- CometHashAggregate (27) + : : +- CometProject (26) + : : +- CometBroadcastHashJoin (25) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (19) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : +- BroadcastExchange (41) + : +- * ColumnarToRow (40) + : +- CometFilter (39) + : +- CometScan parquet spark_catalog.default.customer (38) + +- BroadcastExchange (47) + +- * ColumnarToRow (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.customer_address (44) (1) Scan parquet spark_catalog.default.catalog_returns @@ -61,73 +63,82 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_year#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] + +(7) CometBroadcastHashJoin +Left output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [cr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight -(6) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] +(8) CometProject Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] +Arguments: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3], [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -(7) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_state#8] +(9) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#8, ca_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] ReadSchema: struct -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_state#8] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(ca_state#8)) +(10) CometFilter +Input [2]: [ca_address_sk#8, ca_state#9] +Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_state#9)) -(9) ColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#7, ca_state#8] +(11) CometBroadcastExchange +Input [2]: [ca_address_sk#8, ca_state#9] +Arguments: [ca_address_sk#8, ca_state#9] -(10) BroadcastExchange -Input [2]: [ca_address_sk#7, ca_state#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] +Right output [2]: [ca_address_sk#8, ca_state#9] +Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#8] +(13) CometProject +Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#9] +Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -(13) HashAggregate [codegen id : 3] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] -Keys [2]: [cr_returning_customer_sk#1, ca_state#8] +(14) CometHashAggregate +Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] +Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#9] -Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] -(14) Exchange -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(15) ColumnarToRow [codegen id : 1] +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#10] + +(16) Exchange +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#10] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 11] -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#8] +(17) HashAggregate [codegen id : 7] +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#10] +Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#11] -Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] +Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#9 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] -(16) Filter [codegen id : 11] +(18) Filter [codegen id : 7] Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] Condition : isnotnull(ctr_total_return#14) -(17) Scan parquet spark_catalog.default.catalog_returns +(19) Scan parquet spark_catalog.default.catalog_returns Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Batched: true Location: InMemoryFileIndex [] @@ -135,185 +146,181 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(18) CometFilter +(20) CometFilter Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Condition : isnotnull(cr_returning_addr_sk#2) -(19) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] - -(20) ReusedExchange [Reuses operator id: 53] +(21) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#6] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [cr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight -(22) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] +(23) CometProject Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] +Arguments: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3], [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#7, ca_state#8] +(24) ReusedExchange [Reuses operator id: 11] +Output [2]: [ca_address_sk#8, ca_state#9] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(25) CometBroadcastHashJoin +Left output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] +Right output [2]: [ca_address_sk#8, ca_state#9] +Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight -(25) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#8] +(26) CometProject +Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#9] +Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -(26) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] -Keys [2]: [cr_returning_customer_sk#1, ca_state#8] +(27) CometHashAggregate +Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] +Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#16] -Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] -(27) Exchange -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(28) ColumnarToRow [codegen id : 2] +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#16] -(28) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#17] -Keys [2]: [cr_returning_customer_sk#1, ca_state#8] +(29) Exchange +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#16] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(30) HashAggregate [codegen id : 3] +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#16] +Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#11] -Results [2]: [ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] +Results [2]: [ca_state#9 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] -(29) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 3] Input [2]: [ctr_state#13, ctr_total_return#14] Keys [1]: [ctr_state#13] Functions [1]: [partial_avg(ctr_total_return#14)] -Aggregate Attributes [2]: [sum#18, count#19] -Results [3]: [ctr_state#13, sum#20, count#21] +Aggregate Attributes [2]: [sum#17, count#18] +Results [3]: [ctr_state#13, sum#19, count#20] -(30) Exchange -Input [3]: [ctr_state#13, sum#20, count#21] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(32) Exchange +Input [3]: [ctr_state#13, sum#19, count#20] +Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(31) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#13, sum#20, count#21] +(33) HashAggregate [codegen id : 4] +Input [3]: [ctr_state#13, sum#19, count#20] Keys [1]: [ctr_state#13] Functions [1]: [avg(ctr_total_return#14)] -Aggregate Attributes [1]: [avg(ctr_total_return#14)#22] -Results [2]: [(avg(ctr_total_return#14)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#13 AS ctr_state#13#24] +Aggregate Attributes [1]: [avg(ctr_total_return#14)#21] +Results [2]: [(avg(ctr_total_return#14)#21 * 1.2) AS (avg(ctr_total_return) * 1.2)#22, ctr_state#13 AS ctr_state#13#23] -(32) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) +(34) Filter [codegen id : 4] +Input [2]: [(avg(ctr_total_return) * 1.2)#22, ctr_state#13#23] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#22) -(33) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] +(35) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#22, ctr_state#13#23] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_state#13] -Right keys [1]: [ctr_state#13#24] +Right keys [1]: [ctr_state#13#23] Join type: Inner -Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) +Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#22) -(35) Project [codegen id : 11] +(37) Project [codegen id : 7] Output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#23, ctr_state#13#24] +Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#22, ctr_state#13#23] -(36) Scan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] +(38) Scan parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#24, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(37) CometFilter -Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) +(39) CometFilter +Input [6]: [c_customer_sk#24, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#26)) -(38) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] +(40) ColumnarToRow [codegen id : 5] +Input [6]: [c_customer_sk#24, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29] -(39) BroadcastExchange -Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(41) BroadcastExchange +Input [6]: [c_customer_sk#24, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(40) BroadcastHashJoin [codegen id : 11] +(42) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ctr_customer_sk#12] -Right keys [1]: [c_customer_sk#25] +Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(41) Project [codegen id : 11] -Output [6]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] -Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] +(43) Project [codegen id : 7] +Output [6]: [ctr_total_return#14, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29] +Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#24, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29] -(42) Scan parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] +(44) Scan parquet spark_catalog.default.customer_address +Output [12]: [ca_address_sk#30, ca_street_number#31, ca_street_name#32, ca_street_type#33, ca_suite_number#34, ca_city#35, ca_county#36, ca_state#37, ca_zip#38, ca_country#39, ca_gmt_offset#40, ca_location_type#41] 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 -(43) CometFilter -Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -Condition : ((isnotnull(ca_state#38) AND (ca_state#38 = GA)) AND isnotnull(ca_address_sk#31)) +(45) CometFilter +Input [12]: [ca_address_sk#30, ca_street_number#31, ca_street_name#32, ca_street_type#33, ca_suite_number#34, ca_city#35, ca_county#36, ca_state#37, ca_zip#38, ca_country#39, ca_gmt_offset#40, ca_location_type#41] +Condition : ((isnotnull(ca_state#37) AND (ca_state#37 = GA)) AND isnotnull(ca_address_sk#30)) -(44) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] +(46) ColumnarToRow [codegen id : 6] +Input [12]: [ca_address_sk#30, ca_street_number#31, ca_street_name#32, ca_street_type#33, ca_suite_number#34, ca_city#35, ca_county#36, ca_state#37, ca_zip#38, ca_country#39, ca_gmt_offset#40, ca_location_type#41] -(45) BroadcastExchange -Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(47) BroadcastExchange +Input [12]: [ca_address_sk#30, ca_street_number#31, ca_street_name#32, ca_street_type#33, ca_suite_number#34, ca_city#35, ca_county#36, ca_state#37, ca_zip#38, ca_country#39, ca_gmt_offset#40, ca_location_type#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#27] -Right keys [1]: [ca_address_sk#31] +(48) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#26] +Right keys [1]: [ca_address_sk#30] Join type: Inner Join condition: None -(47) Project [codegen id : 11] -Output [16]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] -Input [18]: [ctr_total_return#14, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] +(49) Project [codegen id : 7] +Output [16]: [c_customer_id#25, c_salutation#27, c_first_name#28, c_last_name#29, ca_street_number#31, ca_street_name#32, ca_street_type#33, ca_suite_number#34, ca_city#35, ca_county#36, ca_state#37, ca_zip#38, ca_country#39, ca_gmt_offset#40, ca_location_type#41, ctr_total_return#14] +Input [18]: [ctr_total_return#14, c_customer_id#25, c_current_addr_sk#26, c_salutation#27, c_first_name#28, c_last_name#29, ca_address_sk#30, ca_street_number#31, ca_street_name#32, ca_street_type#33, ca_suite_number#34, ca_city#35, ca_county#36, ca_state#37, ca_zip#38, ca_country#39, ca_gmt_offset#40, ca_location_type#41] -(48) TakeOrderedAndProject -Input [16]: [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] -Arguments: 100, [c_customer_id#26 ASC NULLS FIRST, c_salutation#28 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, c_last_name#30 ASC NULLS FIRST, ca_street_number#32 ASC NULLS FIRST, ca_street_name#33 ASC NULLS FIRST, ca_street_type#34 ASC NULLS FIRST, ca_suite_number#35 ASC NULLS FIRST, ca_city#36 ASC NULLS FIRST, ca_county#37 ASC NULLS FIRST, ca_state#38 ASC NULLS FIRST, ca_zip#39 ASC NULLS FIRST, ca_country#40 ASC NULLS FIRST, ca_gmt_offset#41 ASC NULLS FIRST, ca_location_type#42 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#26, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42, ctr_total_return#14] +(50) TakeOrderedAndProject +Input [16]: [c_customer_id#25, c_salutation#27, c_first_name#28, c_last_name#29, ca_street_number#31, ca_street_name#32, ca_street_type#33, ca_suite_number#34, ca_city#35, ca_county#36, ca_state#37, ca_zip#38, ca_country#39, ca_gmt_offset#40, ca_location_type#41, ctr_total_return#14] +Arguments: 100, [c_customer_id#25 ASC NULLS FIRST, c_salutation#27 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST, c_last_name#29 ASC NULLS FIRST, ca_street_number#31 ASC NULLS FIRST, ca_street_name#32 ASC NULLS FIRST, ca_street_type#33 ASC NULLS FIRST, ca_suite_number#34 ASC NULLS FIRST, ca_city#35 ASC NULLS FIRST, ca_county#36 ASC NULLS FIRST, ca_state#37 ASC NULLS FIRST, ca_zip#38 ASC NULLS FIRST, ca_country#39 ASC NULLS FIRST, ca_gmt_offset#40 ASC NULLS FIRST, ca_location_type#41 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#25, c_salutation#27, c_first_name#28, c_last_name#29, ca_street_number#31, ca_street_name#32, ca_street_type#33, ca_suite_number#34, ca_city#35, ca_county#36, ca_state#37, ca_zip#38, ca_country#39, ca_gmt_offset#40, ca_location_type#41, ctr_total_return#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (53) -+- * ColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan parquet spark_catalog.default.date_dim (49) +BroadcastExchange (55) ++- * ColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) -(49) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#43] +(51) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_year#7] 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 -(50) CometFilter -Input [2]: [d_date_sk#6, d_year#43] -Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#6)) +(52) CometFilter +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(51) CometProject -Input [2]: [d_date_sk#6, d_year#43] +(53) CometProject +Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(52) ColumnarToRow [codegen id : 1] +(54) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(53) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 17 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt index da163f0233..f80554fb55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -1,5 +1,5 @@ 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 (11) + WholeStageCodegen (7) 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 [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] @@ -10,14 +10,14 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st 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] #1 - WholeStageCodegen (3) - 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] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -28,52 +28,48 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) + BroadcastExchange #5 + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] InputAdapter - Exchange [ctr_state] #5 - WholeStageCodegen (7) + Exchange [ctr_state] #6 + WholeStageCodegen (3) 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] #6 - 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] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [cr_returning_customer_sk,ca_state] #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometFilter [cr_returning_addr_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) + BroadcastExchange #8 + WholeStageCodegen (5) ColumnarToRow InputAdapter CometFilter [c_customer_sk,c_current_addr_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) + BroadcastExchange #9 + WholeStageCodegen (6) ColumnarToRow InputAdapter CometFilter [ca_state,ca_address_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt index 7609fa5208..00e38f6948 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt @@ -1,29 +1,30 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- Exchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * ColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan parquet spark_catalog.default.store_sales (16) +TakeOrderedAndProject (26) ++- * HashAggregate (25) + +- Exchange (24) + +- * ColumnarToRow (23) + +- CometHashAggregate (22) + +- CometProject (21) + +- CometBroadcastHashJoin (20) + :- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- CometBroadcastExchange (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.inventory (4) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.date_dim (10) + +- CometProject (19) + +- CometFilter (18) + +- CometScan parquet spark_catalog.default.store_sales (17) (1) Scan parquet spark_catalog.default.item @@ -41,10 +42,7 @@ Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) A Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(4) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] - -(5) Scan parquet spark_catalog.default.inventory +(4) Scan parquet spark_catalog.default.inventory Output [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Batched: true Location: InMemoryFileIndex [] @@ -52,128 +50,136 @@ PartitionFilters: [isnotnull(inv_date_sk#8), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Condition : (((isnotnull(inv_quantity_on_hand#7) AND (inv_quantity_on_hand#7 >= 100)) AND (inv_quantity_on_hand#7 <= 500)) AND isnotnull(inv_item_sk#6)) -(7) CometProject +(6) CometProject Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] Arguments: [inv_item_sk#6, inv_date_sk#8], [inv_item_sk#6, inv_date_sk#8] -(8) ColumnarToRow [codegen id : 1] -Input [2]: [inv_item_sk#6, inv_date_sk#8] - -(9) BroadcastExchange +(7) CometBroadcastExchange Input [2]: [inv_item_sk#6, inv_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [inv_item_sk#6, inv_date_sk#8] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#6] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Right output [2]: [inv_item_sk#6, inv_date_sk#8] +Arguments: [i_item_sk#1], [inv_item_sk#6], Inner, BuildRight -(11) Project [codegen id : 3] -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] +(9) CometProject Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_item_sk#6, inv_date_sk#8] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] -(12) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#10] +(10) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_date#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [inv_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(11) CometFilter +Input [2]: [d_date_sk#10, d_date#11] +Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-05-25)) AND (d_date#11 <= 2000-07-24)) AND isnotnull(d_date_sk#10)) -(14) Project [codegen id : 3] -Output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +(12) CometProject +Input [2]: [d_date_sk#10, d_date#11] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] + +(14) CometBroadcastHashJoin +Left output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [inv_date_sk#8], [d_date_sk#10], Inner, BuildRight + +(15) CometProject Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8, d_date_sk#10] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(15) BroadcastExchange +(16) CometBroadcastExchange Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] -(16) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] +(17) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_item_sk#12, ss_sold_date_sk#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(17) CometFilter -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_item_sk#11) - -(18) CometProject -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#11], [ss_item_sk#11] +(18) CometFilter +Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_item_sk#12) -(19) ColumnarToRow -Input [1]: [ss_item_sk#11] +(19) CometProject +Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] +Arguments: [ss_item_sk#12], [ss_item_sk#12] -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#11] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Right output [1]: [ss_item_sk#12] +Arguments: [i_item_sk#1], [ss_item_sk#12], Inner, BuildLeft -(21) Project [codegen id : 4] -Output [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, ss_item_sk#11] +(21) CometProject +Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, ss_item_sk#12] +Arguments: [i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_id#2, i_item_desc#3, i_current_price#4] -(22) HashAggregate [codegen id : 4] +(22) CometHashAggregate Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(23) Exchange +(23) ColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(24) HashAggregate [codegen id : 5] +(24) Exchange +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(25) HashAggregate [codegen id : 2] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Functions: [] Aggregate Attributes: [] Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -(25) TakeOrderedAndProject +(26) TakeOrderedAndProject Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== -Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (30) -+- * ColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.date_dim (26) +Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(26) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_date#13] +(27) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_date#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter -Input [2]: [d_date_sk#10, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-05-25)) AND (d_date#13 <= 2000-07-24)) AND isnotnull(d_date_sk#10)) +(28) CometFilter +Input [2]: [d_date_sk#10, d_date#11] +Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-05-25)) AND (d_date#11 <= 2000-07-24)) AND isnotnull(d_date_sk#10)) -(28) CometProject -Input [2]: [d_date_sk#10, d_date#13] +(29) CometProject +Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(29) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt index 0252eb575f..a4d96e6f14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt @@ -1,44 +1,38 @@ TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_current_price] InputAdapter Exchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_current_price,i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_quantity_on_hand,inv_item_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometProject [i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] + CometBroadcastExchange #2 + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + CometBroadcastHashJoin [i_item_sk,inv_item_sk] + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometBroadcastExchange #3 + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [ss_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 0aa5212b01..1838b9a777 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,50 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (46) -+- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_returns (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (10) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- BroadcastExchange (43) - +- * HashAggregate (42) - +- Exchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometFilter (32) - : : +- CometScan parquet spark_catalog.default.web_returns (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * HashAggregate (19) + : : +- Exchange (18) + : : +- * ColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometProject (15) + : : +- CometBroadcastHashJoin (14) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_returns (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.item (3) + : : +- CometBroadcastExchange (13) + : : +- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- ReusedExchange (10) + : +- BroadcastExchange (32) + : +- * HashAggregate (31) + : +- Exchange (30) + : +- * ColumnarToRow (29) + : +- CometHashAggregate (28) + : +- CometProject (27) + : +- CometBroadcastHashJoin (26) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (20) + : : +- ReusedExchange (22) + : +- ReusedExchange (25) + +- BroadcastExchange (47) + +- * HashAggregate (46) + +- Exchange (45) + +- * ColumnarToRow (44) + +- CometHashAggregate (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.web_returns (35) + : +- ReusedExchange (37) + +- ReusedExchange (40) (1) Scan parquet spark_catalog.default.store_returns @@ -59,142 +63,155 @@ ReadSchema: struct Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) -(3) ColumnarToRow [codegen id : 5] -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#5, i_item_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [i_item_sk#5, i_item_id#6] Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [2]: [i_item_sk#5, i_item_id#6] +Arguments: [i_item_sk#5, i_item_id#6] -(7) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] +Right output [2]: [i_item_sk#5, i_item_id#6] +Arguments: [sr_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) Project [codegen id : 5] -Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] +(7) CometProject Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#6] +Arguments: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6], [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] + +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_date#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct -(10) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#7] +(9) CometFilter +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(10) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date#9] + +(11) CometBroadcastHashJoin +Left output [2]: [d_date_sk#7, d_date#8] +Right output [1]: [d_date#9] +Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight -(12) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#6] +(12) CometProject +Input [2]: [d_date_sk#7, d_date#8] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(14) CometBroadcastHashJoin +Left output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] +Right output [1]: [d_date_sk#7] +Arguments: [sr_returned_date_sk#3], [d_date_sk#7], Inner, BuildRight + +(15) CometProject Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6, d_date_sk#7] +Arguments: [sr_return_quantity#2, i_item_id#6], [sr_return_quantity#2, i_item_id#6] -(13) HashAggregate [codegen id : 5] +(16) CometHashAggregate Input [2]: [sr_return_quantity#2, i_item_id#6] Keys [1]: [i_item_id#6] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#8] -Results [2]: [i_item_id#6, sum#9] -(14) Exchange -Input [2]: [i_item_id#6, sum#9] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(17) ColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#6, sum#10] + +(18) Exchange +Input [2]: [i_item_id#6, sum#10] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#6, sum#9] +(19) HashAggregate [codegen id : 6] +Input [2]: [i_item_id#6, sum#10] Keys [1]: [i_item_id#6] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#10] -Results [2]: [i_item_id#6 AS item_id#11, sum(sr_return_quantity#2)#10 AS sr_item_qty#12] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#11] +Results [2]: [i_item_id#6 AS item_id#12, sum(sr_return_quantity#2)#11 AS sr_item_qty#13] -(16) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] +(20) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#15), dynamicpruningexpression(cr_returned_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cr_returned_date_sk#16), dynamicpruningexpression(cr_returned_date_sk#16 IN dynamicpruning#17)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(17) CometFilter -Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] -Condition : isnotnull(cr_item_sk#13) +(21) CometFilter +Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] +Condition : isnotnull(cr_item_sk#14) -(18) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] +(22) ReusedExchange [Reuses operator id: 5] +Output [2]: [i_item_sk#18, i_item_id#19] -(19) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#17, i_item_id#18] +(23) CometBroadcastHashJoin +Left output [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] +Right output [2]: [i_item_sk#18, i_item_id#19] +Arguments: [cr_item_sk#14], [i_item_sk#18], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#13] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(24) CometProject +Input [5]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16, i_item_sk#18, i_item_id#19] +Arguments: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19], [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19] -(21) Project [codegen id : 10] -Output [3]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18] -Input [5]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15, i_item_sk#17, i_item_id#18] +(25) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#20] -(22) ReusedExchange [Reuses operator id: 62] -Output [1]: [d_date_sk#19] +(26) CometBroadcastHashJoin +Left output [3]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19] +Right output [1]: [d_date_sk#20] +Arguments: [cr_returned_date_sk#16], [d_date_sk#20], Inner, BuildRight -(23) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#15] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None +(27) CometProject +Input [4]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19, d_date_sk#20] +Arguments: [cr_return_quantity#15, i_item_id#19], [cr_return_quantity#15, i_item_id#19] + +(28) CometHashAggregate +Input [2]: [cr_return_quantity#15, i_item_id#19] +Keys [1]: [i_item_id#19] +Functions [1]: [partial_sum(cr_return_quantity#15)] -(24) Project [codegen id : 10] -Output [2]: [cr_return_quantity#14, i_item_id#18] -Input [4]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18, d_date_sk#19] - -(25) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#14, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(cr_return_quantity#14)] -Aggregate Attributes [1]: [sum#20] -Results [2]: [i_item_id#18, sum#21] - -(26) Exchange -Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#18, sum#21] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(cr_return_quantity#14)] -Aggregate Attributes [1]: [sum(cr_return_quantity#14)#22] -Results [2]: [i_item_id#18 AS item_id#23, sum(cr_return_quantity#14)#22 AS cr_item_qty#24] - -(28) BroadcastExchange +(29) ColumnarToRow [codegen id : 2] +Input [2]: [i_item_id#19, sum#21] + +(30) Exchange +Input [2]: [i_item_id#19, sum#21] +Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(31) HashAggregate [codegen id : 3] +Input [2]: [i_item_id#19, sum#21] +Keys [1]: [i_item_id#19] +Functions [1]: [sum(cr_return_quantity#15)] +Aggregate Attributes [1]: [sum(cr_return_quantity#15)#22] +Results [2]: [i_item_id#19 AS item_id#23, sum(cr_return_quantity#15)#22 AS cr_item_qty#24] + +(32) BroadcastExchange Input [2]: [item_id#23, cr_item_qty#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(29) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#11] +(33) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [item_id#12] Right keys [1]: [item_id#23] Join type: Inner Join condition: None -(30) Project [codegen id : 18] -Output [3]: [item_id#11, sr_item_qty#12, cr_item_qty#24] -Input [4]: [item_id#11, sr_item_qty#12, item_id#23, cr_item_qty#24] +(34) Project [codegen id : 6] +Output [3]: [item_id#12, sr_item_qty#13, cr_item_qty#24] +Input [4]: [item_id#12, sr_item_qty#13, item_id#23, cr_item_qty#24] -(31) Scan parquet spark_catalog.default.web_returns +(35) Scan parquet spark_catalog.default.web_returns Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] Batched: true Location: InMemoryFileIndex [] @@ -202,170 +219,156 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(32) CometFilter +(36) CometFilter Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] Condition : isnotnull(wr_item_sk#25) -(33) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 7] +(37) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#29, i_item_id#30] -(35) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#25] -Right keys [1]: [i_item_sk#29] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] +Right output [2]: [i_item_sk#29, i_item_id#30] +Arguments: [wr_item_sk#25], [i_item_sk#29], Inner, BuildRight -(36) Project [codegen id : 16] -Output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] +(39) CometProject Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#29, i_item_id#30] +Arguments: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30], [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] -(37) ReusedExchange [Reuses operator id: 62] +(40) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#31] -(38) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#27] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None +(41) CometBroadcastHashJoin +Left output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] +Right output [1]: [d_date_sk#31] +Arguments: [wr_returned_date_sk#27], [d_date_sk#31], Inner, BuildRight -(39) Project [codegen id : 16] -Output [2]: [wr_return_quantity#26, i_item_id#30] +(42) CometProject Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30, d_date_sk#31] +Arguments: [wr_return_quantity#26, i_item_id#30], [wr_return_quantity#26, i_item_id#30] -(40) HashAggregate [codegen id : 16] +(43) CometHashAggregate Input [2]: [wr_return_quantity#26, i_item_id#30] Keys [1]: [i_item_id#30] Functions [1]: [partial_sum(wr_return_quantity#26)] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#30, sum#33] -(41) Exchange -Input [2]: [i_item_id#30, sum#33] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(44) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_id#30, sum#32] + +(45) Exchange +Input [2]: [i_item_id#30, sum#32] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(42) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#30, sum#33] +(46) HashAggregate [codegen id : 5] +Input [2]: [i_item_id#30, sum#32] Keys [1]: [i_item_id#30] Functions [1]: [sum(wr_return_quantity#26)] -Aggregate Attributes [1]: [sum(wr_return_quantity#26)#34] -Results [2]: [i_item_id#30 AS item_id#35, sum(wr_return_quantity#26)#34 AS wr_item_qty#36] +Aggregate Attributes [1]: [sum(wr_return_quantity#26)#33] +Results [2]: [i_item_id#30 AS item_id#34, sum(wr_return_quantity#26)#33 AS wr_item_qty#35] -(43) BroadcastExchange -Input [2]: [item_id#35, wr_item_qty#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] +(47) BroadcastExchange +Input [2]: [item_id#34, wr_item_qty#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#11] -Right keys [1]: [item_id#35] +(48) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [item_id#12] +Right keys [1]: [item_id#34] Join type: Inner Join condition: None -(45) Project [codegen id : 18] -Output [8]: [item_id#11, sr_item_qty#12, (((cast(sr_item_qty#12 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS sr_dev#37, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS cr_dev#38, wr_item_qty#36, (((cast(wr_item_qty#36 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS wr_dev#39, (cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as decimal(20,0)) / 3.0) AS average#40] -Input [5]: [item_id#11, sr_item_qty#12, cr_item_qty#24, item_id#35, wr_item_qty#36] +(49) Project [codegen id : 6] +Output [8]: [item_id#12, sr_item_qty#13, (((cast(sr_item_qty#13 as double) / cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)) / 3.0) * 100.0) AS sr_dev#36, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)) / 3.0) * 100.0) AS cr_dev#37, wr_item_qty#35, (((cast(wr_item_qty#35 as double) / cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)) / 3.0) * 100.0) AS wr_dev#38, (cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as decimal(20,0)) / 3.0) AS average#39] +Input [5]: [item_id#12, sr_item_qty#13, cr_item_qty#24, item_id#34, wr_item_qty#35] -(46) TakeOrderedAndProject -Input [8]: [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] -Arguments: 100, [item_id#11 ASC NULLS FIRST, sr_item_qty#12 ASC NULLS FIRST], [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] +(50) TakeOrderedAndProject +Input [8]: [item_id#12, sr_item_qty#13, sr_dev#36, cr_item_qty#24, cr_dev#37, wr_item_qty#35, wr_dev#38, average#39] +Arguments: 100, [item_id#12 ASC NULLS FIRST, sr_item_qty#13 ASC NULLS FIRST], [item_id#12, sr_item_qty#13, sr_dev#36, cr_item_qty#24, cr_dev#37, wr_item_qty#35, wr_dev#38, average#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * Project (61) - +- * BroadcastHashJoin LeftSemi BuildRight (60) - :- * ColumnarToRow (49) - : +- CometFilter (48) - : +- CometScan parquet spark_catalog.default.date_dim (47) - +- BroadcastExchange (59) - +- * Project (58) - +- * BroadcastHashJoin LeftSemi BuildRight (57) - :- * ColumnarToRow (51) - : +- CometScan parquet spark_catalog.default.date_dim (50) - +- BroadcastExchange (56) - +- * ColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan parquet spark_catalog.default.date_dim (52) - - -(47) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_date#41] +BroadcastExchange (64) ++- * ColumnarToRow (63) + +- CometProject (62) + +- CometBroadcastHashJoin (61) + :- CometFilter (52) + : +- CometScan parquet spark_catalog.default.date_dim (51) + +- CometBroadcastExchange (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometScan parquet spark_catalog.default.date_dim (53) + +- CometBroadcastExchange (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) + + +(51) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(48) CometFilter -Input [2]: [d_date_sk#7, d_date#41] +(52) CometFilter +Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(49) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#7, d_date#41] - -(50) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#42, d_week_seq#43] +(53) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#9, d_week_seq#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(51) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#42, d_week_seq#43] - -(52) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date#44, d_week_seq#45] +(54) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#41, d_week_seq#42] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct -(53) CometFilter -Input [2]: [d_date#44, d_week_seq#45] -Condition : cast(d_date#44 as string) IN (2000-06-30,2000-09-27,2000-11-17) +(55) CometFilter +Input [2]: [d_date#41, d_week_seq#42] +Condition : cast(d_date#41 as string) IN (2000-06-30,2000-09-27,2000-11-17) -(54) CometProject -Input [2]: [d_date#44, d_week_seq#45] -Arguments: [d_week_seq#45], [d_week_seq#45] +(56) CometProject +Input [2]: [d_date#41, d_week_seq#42] +Arguments: [d_week_seq#42], [d_week_seq#42] -(55) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#45] +(57) CometBroadcastExchange +Input [1]: [d_week_seq#42] +Arguments: [d_week_seq#42] -(56) BroadcastExchange -Input [1]: [d_week_seq#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +(58) CometBroadcastHashJoin +Left output [2]: [d_date#9, d_week_seq#40] +Right output [1]: [d_week_seq#42] +Arguments: [d_week_seq#40], [d_week_seq#42], LeftSemi, BuildRight -(57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#43] -Right keys [1]: [d_week_seq#45] -Join type: LeftSemi -Join condition: None +(59) CometProject +Input [2]: [d_date#9, d_week_seq#40] +Arguments: [d_date#9], [d_date#9] -(58) Project [codegen id : 2] -Output [1]: [d_date#42] -Input [2]: [d_date#42, d_week_seq#43] +(60) CometBroadcastExchange +Input [1]: [d_date#9] +Arguments: [d_date#9] -(59) BroadcastExchange -Input [1]: [d_date#42] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=8] +(61) CometBroadcastHashJoin +Left output [2]: [d_date_sk#7, d_date#8] +Right output [1]: [d_date#9] +Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight -(60) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date#41] -Right keys [1]: [d_date#42] -Join type: LeftSemi -Join condition: None +(62) CometProject +Input [2]: [d_date_sk#7, d_date#8] +Arguments: [d_date_sk#7], [d_date_sk#7] -(61) Project [codegen id : 3] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#41] +(63) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] -(62) BroadcastExchange +(64) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 16 Hosting Expression = cr_returned_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#16 IN dynamicpruning#4 -Subquery:3 Hosting operator id = 31 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 35 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index a8f1ba3f10..ef7d35e21e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - WholeStageCodegen (18) + WholeStageCodegen (6) Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] BroadcastHashJoin [item_id,item_id] Project [item_id,sr_item_qty,cr_item_qty] @@ -7,89 +7,77 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] InputAdapter Exchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - Project [sr_return_quantity,i_item_id] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [sr_return_quantity,sr_returned_date_sk,i_item_id] - BroadcastHashJoin [sr_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,sr_return_quantity] + CometProject [sr_return_quantity,i_item_id] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] + CometProject [sr_return_quantity,sr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [sr_item_sk,i_item_sk] CometFilter [sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (3) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date,d_date] CometFilter [d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - Project [d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date] - CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #3 + CometProject [d_date] + CometBroadcastHashJoin [d_week_seq,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange #4 + CometProject [d_week_seq] + CometFilter [d_date] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange #5 + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date,d_date] + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + ReusedExchange [d_date] #3 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (11) + BroadcastExchange #7 + WholeStageCodegen (3) HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] InputAdapter - Exchange [i_item_id] #7 - WholeStageCodegen (10) - HashAggregate [i_item_id,cr_return_quantity] [sum,sum] - Project [cr_return_quantity,i_item_id] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cr_return_quantity,cr_returned_date_sk,i_item_id] - BroadcastHashJoin [cr_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,cr_return_quantity] + CometProject [cr_return_quantity,i_item_id] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometProject [cr_return_quantity,cr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [cr_item_sk,i_item_sk] CometFilter [cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #6 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (17) + BroadcastExchange #9 + WholeStageCodegen (5) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] InputAdapter - Exchange [i_item_id] #9 - WholeStageCodegen (16) - HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - Project [wr_return_quantity,i_item_id] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Project [wr_return_quantity,wr_returned_date_sk,i_item_id] - BroadcastHashJoin [wr_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [i_item_id] #10 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,wr_return_quantity] + CometProject [wr_return_quantity,i_item_id] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] + CometProject [wr_return_quantity,wr_returned_date_sk,i_item_id] + CometBroadcastHashJoin [wr_item_sk,i_item_sk] CometFilter [wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt index 8dc935d1da..468af40d20 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt @@ -1,41 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- * BroadcastHashJoin Inner BuildLeft (35) - :- BroadcastExchange (30) - : +- * Project (29) - : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.customer_address (4) - : : : +- BroadcastExchange (14) - : : : +- * ColumnarToRow (13) - : : : +- CometFilter (12) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (11) - : : +- BroadcastExchange (20) - : : +- * ColumnarToRow (19) - : : +- CometFilter (18) - : : +- CometScan parquet spark_catalog.default.household_demographics (17) - : +- BroadcastExchange (27) - : +- * ColumnarToRow (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.income_band (23) - +- * ColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.store_returns (31) +TakeOrderedAndProject (32) ++- * Project (31) + +- * ColumnarToRow (30) + +- CometBroadcastHashJoin (29) + :- CometBroadcastExchange (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.customer_address (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.household_demographics (14) + : +- CometBroadcastExchange (22) + : +- CometProject (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.income_band (19) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.store_returns (26) (1) Scan parquet spark_catalog.default.customer @@ -49,162 +44,142 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [ca_address_sk#7, ca_city#8] Condition : ((isnotnull(ca_city#8) AND (ca_city#8 = Edgewood)) AND isnotnull(ca_address_sk#7)) -(6) CometProject +(5) CometProject Input [2]: [ca_address_sk#7, ca_city#8] Arguments: [ca_address_sk#7], [ca_address_sk#7] -(7) ColumnarToRow [codegen id : 1] -Input [1]: [ca_address_sk#7] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ca_address_sk#7] -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#4] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] +Right output [1]: [ca_address_sk#7] +Arguments: [c_current_addr_sk#4], [ca_address_sk#7], Inner, BuildRight -(10) Project [codegen id : 5] -Output [5]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6] +(8) CometProject Input [7]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6, ca_address_sk#7] +Arguments: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6], [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6] -(11) Scan parquet spark_catalog.default.customer_demographics +(9) Scan parquet spark_catalog.default.customer_demographics Output [1]: [cd_demo_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [1]: [cd_demo_sk#9] Condition : isnotnull(cd_demo_sk#9) -(13) ColumnarToRow [codegen id : 2] -Input [1]: [cd_demo_sk#9] - -(14) BroadcastExchange +(11) CometBroadcastExchange Input [1]: [cd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [cd_demo_sk#9] -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#9] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [5]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6] +Right output [1]: [cd_demo_sk#9] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#9], Inner, BuildRight -(16) Project [codegen id : 5] -Output [5]: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9] +(13) CometProject Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9] +Arguments: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9], [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9] -(17) Scan parquet spark_catalog.default.household_demographics +(14) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#10, hd_income_band_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] ReadSchema: struct -(18) CometFilter +(15) CometFilter Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] Condition : (isnotnull(hd_demo_sk#10) AND isnotnull(hd_income_band_sk#11)) -(19) ColumnarToRow [codegen id : 3] +(16) CometBroadcastExchange Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] +Arguments: [hd_demo_sk#10, hd_income_band_sk#11] -(20) BroadcastExchange -Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [5]: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9] +Right output [2]: [hd_demo_sk#10, hd_income_band_sk#11] +Arguments: [c_current_hdemo_sk#3], [hd_demo_sk#10], Inner, BuildRight -(22) Project [codegen id : 5] -Output [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11] +(18) CometProject Input [7]: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_demo_sk#10, hd_income_band_sk#11] +Arguments: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11], [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11] -(23) Scan parquet spark_catalog.default.income_band +(19) Scan parquet spark_catalog.default.income_band Output [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] Batched: true Location [not included in comparison]/{warehouse_dir}/income_band] PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] ReadSchema: struct -(24) CometFilter +(20) CometFilter Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] Condition : ((((isnotnull(ib_lower_bound#13) AND isnotnull(ib_upper_bound#14)) AND (ib_lower_bound#13 >= 38128)) AND (ib_upper_bound#14 <= 88128)) AND isnotnull(ib_income_band_sk#12)) -(25) CometProject +(21) CometProject Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] Arguments: [ib_income_band_sk#12], [ib_income_band_sk#12] -(26) ColumnarToRow [codegen id : 4] +(22) CometBroadcastExchange Input [1]: [ib_income_band_sk#12] +Arguments: [ib_income_band_sk#12] -(27) BroadcastExchange -Input [1]: [ib_income_band_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(23) CometBroadcastHashJoin +Left output [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11] +Right output [1]: [ib_income_band_sk#12] +Arguments: [hd_income_band_sk#11], [ib_income_band_sk#12], Inner, BuildRight -(28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [hd_income_band_sk#11] -Right keys [1]: [ib_income_band_sk#12] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 5] -Output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] +(24) CometProject Input [6]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11, ib_income_band_sk#12] +Arguments: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9], [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] -(30) BroadcastExchange +(25) CometBroadcastExchange Input [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [plan_id=5] +Arguments: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] -(31) Scan parquet spark_catalog.default.store_returns +(26) Scan parquet spark_catalog.default.store_returns Output [2]: [sr_cdemo_sk#15, sr_returned_date_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_cdemo_sk)] ReadSchema: struct -(32) CometFilter +(27) CometFilter Input [2]: [sr_cdemo_sk#15, sr_returned_date_sk#16] Condition : isnotnull(sr_cdemo_sk#15) -(33) CometProject +(28) CometProject Input [2]: [sr_cdemo_sk#15, sr_returned_date_sk#16] Arguments: [sr_cdemo_sk#15], [sr_cdemo_sk#15] -(34) ColumnarToRow -Input [1]: [sr_cdemo_sk#15] +(29) CometBroadcastHashJoin +Left output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] +Right output [1]: [sr_cdemo_sk#15] +Arguments: [cd_demo_sk#9], [sr_cdemo_sk#15], Inner, BuildLeft -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cd_demo_sk#9] -Right keys [1]: [sr_cdemo_sk#15] -Join type: Inner -Join condition: None +(30) ColumnarToRow [codegen id : 1] +Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, sr_cdemo_sk#15] -(36) Project [codegen id : 6] +(31) Project [codegen id : 1] Output [3]: [c_customer_id#1 AS customer_id#17, concat(c_last_name#6, , , c_first_name#5) AS customername#18, c_customer_id#1] Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, sr_cdemo_sk#15] -(37) TakeOrderedAndProject +(32) TakeOrderedAndProject Input [3]: [customer_id#17, customername#18, c_customer_id#1] Arguments: 100, [c_customer_id#1 ASC NULLS FIRST], [customer_id#17, customername#18] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt index be3451d29c..fe7f7a2071 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt @@ -1,54 +1,34 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] - WholeStageCodegen (6) + WholeStageCodegen (1) Project [c_customer_id,c_last_name,c_first_name] - BroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] + ColumnarToRow InputAdapter - BroadcastExchange #1 - WholeStageCodegen (5) - Project [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] - Project [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_city,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [ib_income_band_sk] - CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - ColumnarToRow - InputAdapter + CometBroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange #1 + CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] + CometBroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] + CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastExchange #2 + CometProject [ca_address_sk] + CometFilter [ca_city,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange #3 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange #4 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + CometBroadcastExchange #5 + CometProject [ib_income_band_sk] + CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] CometProject [sr_cdemo_sk] CometFilter [sr_cdemo_sk] CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index ce1cc22629..cdd66620ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -1,51 +1,49 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (34) - : : +- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Project (27) - : : : +- * BroadcastHashJoin Inner BuildRight (26) - : : : :- * Project (21) - : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometBroadcastExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.web_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * ColumnarToRow (12) - : : : : : +- CometFilter (11) - : : : : : +- CometScan parquet spark_catalog.default.web_page (10) - : : : : +- BroadcastExchange (19) - : : : : +- * ColumnarToRow (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (16) - : : : +- BroadcastExchange (25) - : : : +- * ColumnarToRow (24) - : : : +- CometFilter (23) - : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : +- BroadcastExchange (32) - : : +- * ColumnarToRow (31) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometScan parquet spark_catalog.default.customer_address (28) - : +- ReusedExchange (35) - +- BroadcastExchange (41) - +- * ColumnarToRow (40) - +- CometFilter (39) - +- CometScan parquet spark_catalog.default.reason (38) +TakeOrderedAndProject (45) ++- * HashAggregate (44) + +- Exchange (43) + +- * ColumnarToRow (42) + +- CometHashAggregate (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometProject (18) + : : : : +- CometBroadcastHashJoin (17) + : : : : :- CometProject (13) + : : : : : +- CometBroadcastHashJoin (12) + : : : : : :- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometBroadcastExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.web_returns (4) + : : : : : +- CometBroadcastExchange (11) + : : : : : +- CometFilter (10) + : : : : : +- CometScan parquet spark_catalog.default.web_page (9) + : : : : +- CometBroadcastExchange (16) + : : : : +- CometFilter (15) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (14) + : : : +- CometBroadcastExchange (21) + : : : +- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (19) + : : +- CometBroadcastExchange (27) + : : +- CometProject (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.customer_address (24) + : +- CometBroadcastExchange (33) + : +- CometProject (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.date_dim (30) + +- CometBroadcastExchange (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.reason (36) (1) Scan parquet spark_catalog.default.web_sales @@ -82,224 +80,217 @@ Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_re (7) CometBroadcastHashJoin Left output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] Right output [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner +Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner, BuildLeft (8) CometProject Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] Arguments: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -(9) ColumnarToRow [codegen id : 7] -Input [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(10) Scan parquet spark_catalog.default.web_page +(9) Scan parquet spark_catalog.default.web_page Output [1]: [wp_web_page_sk#18] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [1]: [wp_web_page_sk#18] Condition : isnotnull(wp_web_page_sk#18) -(12) ColumnarToRow [codegen id : 1] -Input [1]: [wp_web_page_sk#18] - -(13) BroadcastExchange +(11) CometBroadcastExchange Input [1]: [wp_web_page_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [wp_web_page_sk#18] -(14) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ws_web_page_sk#2] -Right keys [1]: [wp_web_page_sk#18] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Right output [1]: [wp_web_page_sk#18] +Arguments: [ws_web_page_sk#2], [wp_web_page_sk#18], Inner, BuildRight -(15) Project [codegen id : 7] -Output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +(13) CometProject Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] +Arguments: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -(16) Scan parquet spark_catalog.default.customer_demographics +(14) Scan parquet spark_catalog.default.customer_demographics Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct -(17) CometFilter +(15) CometFilter Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(cd_marital_status#20)) AND isnotnull(cd_education_status#21)) AND ((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree )) OR ((cd_marital_status#20 = S) AND (cd_education_status#21 = College ))) OR ((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree )))) -(18) ColumnarToRow [codegen id : 2] -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] - -(19) BroadcastExchange +(16) CometBroadcastExchange Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -(20) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [wr_refunded_cdemo_sk#10] -Right keys [1]: [cd_demo_sk#19] -Join type: Inner -Join condition: ((((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#20 = S) AND (cd_education_status#21 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) +(17) CometBroadcastHashJoin +Left output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Right output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Arguments: [wr_refunded_cdemo_sk#10], [cd_demo_sk#19], Inner, ((((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#20 = S) AND (cd_education_status#21 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))), BuildRight -(21) Project [codegen id : 7] -Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21] +(18) CometProject Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21] -(22) Scan parquet spark_catalog.default.customer_demographics +(19) Scan parquet spark_catalog.default.customer_demographics Output [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] ReadSchema: struct -(23) CometFilter +(20) CometFilter Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] Condition : ((isnotnull(cd_demo_sk#22) AND isnotnull(cd_marital_status#23)) AND isnotnull(cd_education_status#24)) -(24) ColumnarToRow [codegen id : 3] +(21) CometBroadcastExchange Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] +Arguments: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -(25) BroadcastExchange -Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], input[1, string, false], input[2, string, false]),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 7] -Left keys [3]: [wr_returning_cdemo_sk#12, cd_marital_status#20, cd_education_status#21] -Right keys [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21] +Right output [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] +Arguments: [wr_returning_cdemo_sk#12, cd_marital_status#20, cd_education_status#21], [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24], Inner, BuildRight -(27) Project [codegen id : 7] -Output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +(23) CometProject Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21, cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] +Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -(28) Scan parquet spark_catalog.default.customer_address +(24) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#25, ca_state#26, ca_country#27] 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, [IN,NJ,OH]),In(ca_state, [CT,KY,WI])),In(ca_state, [AR,IA,LA]))] ReadSchema: struct -(29) CometFilter +(25) CometFilter Input [3]: [ca_address_sk#25, ca_state#26, ca_country#27] Condition : (((isnotnull(ca_country#27) AND (ca_country#27 = United States)) AND isnotnull(ca_address_sk#25)) AND ((ca_state#26 IN (IN,OH,NJ) OR ca_state#26 IN (WI,CT,KY)) OR ca_state#26 IN (LA,IA,AR))) -(30) CometProject +(26) CometProject Input [3]: [ca_address_sk#25, ca_state#26, ca_country#27] Arguments: [ca_address_sk#25, ca_state#26], [ca_address_sk#25, ca_state#26] -(31) ColumnarToRow [codegen id : 4] +(27) CometBroadcastExchange Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: [ca_address_sk#25, ca_state#26] -(32) BroadcastExchange -Input [2]: [ca_address_sk#25, ca_state#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(28) CometBroadcastHashJoin +Left output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Right output [2]: [ca_address_sk#25, ca_state#26] +Arguments: [wr_refunded_addr_sk#11], [ca_address_sk#25], Inner, ((((ca_state#26 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#26 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#26 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))), BuildRight -(33) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [wr_refunded_addr_sk#11] -Right keys [1]: [ca_address_sk#25] -Join type: Inner -Join condition: ((((ca_state#26 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#26 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#26 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))) - -(34) Project [codegen id : 7] -Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +(29) CometProject Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#25, ca_state#26] +Arguments: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] + +(30) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] +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 + +(31) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : ((isnotnull(d_year#29) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) -(35) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#28] +(32) CometProject +Input [2]: [d_date_sk#28, d_year#29] +Arguments: [d_date_sk#28], [d_date_sk#28] + +(33) CometBroadcastExchange +Input [1]: [d_date_sk#28] +Arguments: [d_date_sk#28] -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ws_sold_date_sk#7] -Right keys [1]: [d_date_sk#28] -Join type: Inner -Join condition: None +(34) CometBroadcastHashJoin +Left output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Right output [1]: [d_date_sk#28] +Arguments: [ws_sold_date_sk#7], [d_date_sk#28], Inner, BuildRight -(37) Project [codegen id : 7] -Output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +(35) CometProject Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#28] +Arguments: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -(38) Scan parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#29, r_reason_desc#30] +(36) Scan parquet spark_catalog.default.reason +Output [2]: [r_reason_sk#30, r_reason_desc#31] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] PushedFilters: [IsNotNull(r_reason_sk)] ReadSchema: struct -(39) CometFilter -Input [2]: [r_reason_sk#29, r_reason_desc#30] -Condition : isnotnull(r_reason_sk#29) +(37) CometFilter +Input [2]: [r_reason_sk#30, r_reason_desc#31] +Condition : isnotnull(r_reason_sk#30) -(40) ColumnarToRow [codegen id : 6] -Input [2]: [r_reason_sk#29, r_reason_desc#30] +(38) CometBroadcastExchange +Input [2]: [r_reason_sk#30, r_reason_desc#31] +Arguments: [r_reason_sk#30, r_reason_desc#31] -(41) BroadcastExchange -Input [2]: [r_reason_sk#29, r_reason_desc#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Right output [2]: [r_reason_sk#30, r_reason_desc#31] +Arguments: [wr_reason_sk#13], [r_reason_sk#30], Inner, BuildRight -(42) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [wr_reason_sk#13] -Right keys [1]: [r_reason_sk#29] -Join type: Inner -Join condition: None +(40) CometProject +Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#30, r_reason_desc#31] +Arguments: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31], [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31] -(43) Project [codegen id : 7] -Output [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#30] -Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#29, r_reason_desc#30] - -(44) HashAggregate [codegen id : 7] -Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#30] -Keys [1]: [r_reason_desc#30] +(41) CometHashAggregate +Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31] +Keys [1]: [r_reason_desc#31] Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] -Aggregate Attributes [6]: [sum#31, count#32, sum#33, count#34, sum#35, count#36] -Results [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] -(45) Exchange -Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Arguments: hashpartitioning(r_reason_desc#30, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(42) ColumnarToRow [codegen id : 1] +Input [7]: [r_reason_desc#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] + +(43) Exchange +Input [7]: [r_reason_desc#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] +Arguments: hashpartitioning(r_reason_desc#31, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(46) HashAggregate [codegen id : 8] -Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Keys [1]: [r_reason_desc#30] +(44) HashAggregate [codegen id : 2] +Input [7]: [r_reason_desc#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] +Keys [1]: [r_reason_desc#31] Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] -Aggregate Attributes [3]: [avg(ws_quantity#4)#43, avg(UnscaledValue(wr_refunded_cash#16))#44, avg(UnscaledValue(wr_fee#15))#45] -Results [4]: [substr(r_reason_desc#30, 1, 20) AS substr(r_reason_desc, 1, 20)#46, avg(ws_quantity#4)#43 AS avg(ws_quantity)#47, cast((avg(UnscaledValue(wr_refunded_cash#16))#44 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#48, cast((avg(UnscaledValue(wr_fee#15))#45 / 100.0) as decimal(11,6)) AS avg(wr_fee)#49] +Aggregate Attributes [3]: [avg(ws_quantity#4)#38, avg(UnscaledValue(wr_refunded_cash#16))#39, avg(UnscaledValue(wr_fee#15))#40] +Results [4]: [substr(r_reason_desc#31, 1, 20) AS substr(r_reason_desc, 1, 20)#41, avg(ws_quantity#4)#38 AS avg(ws_quantity)#42, cast((avg(UnscaledValue(wr_refunded_cash#16))#39 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#43, cast((avg(UnscaledValue(wr_fee#15))#40 / 100.0) as decimal(11,6)) AS avg(wr_fee)#44] -(47) TakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] -Arguments: 100, [substr(r_reason_desc, 1, 20)#46 ASC NULLS FIRST, avg(ws_quantity)#47 ASC NULLS FIRST, avg(wr_refunded_cash)#48 ASC NULLS FIRST, avg(wr_fee)#49 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] +(45) TakeOrderedAndProject +Input [4]: [substr(r_reason_desc, 1, 20)#41, avg(ws_quantity)#42, avg(wr_refunded_cash)#43, avg(wr_fee)#44] +Arguments: 100, [substr(r_reason_desc, 1, 20)#41 ASC NULLS FIRST, avg(ws_quantity)#42 ASC NULLS FIRST, avg(wr_refunded_cash)#43 ASC NULLS FIRST, avg(wr_fee)#44 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#41, avg(ws_quantity)#42, avg(wr_refunded_cash)#43, avg(wr_fee)#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(48) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#50] +(46) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] 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 -(49) CometFilter -Input [2]: [d_date_sk#28, d_year#50] -Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2000)) AND isnotnull(d_date_sk#28)) +(47) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : ((isnotnull(d_year#29) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) -(50) CometProject -Input [2]: [d_date_sk#28, d_year#50] +(48) CometProject +Input [2]: [d_date_sk#28, d_year#29] Arguments: [d_date_sk#28], [d_date_sk#28] -(51) ColumnarToRow [codegen id : 1] +(49) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] -(52) BroadcastExchange +(50) BroadcastExchange Input [1]: [d_date_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt index 7c5ee727b7..ecae29c785 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] InputAdapter Exchange [r_reason_desc] #1 - WholeStageCodegen (7) - HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - BroadcastHashJoin [wr_reason_sk,r_reason_sk] - Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] - BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] - Project [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] + CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] + CometBroadcastHashJoin [wr_reason_sk,r_reason_sk] + CometProject [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + CometProject [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] CometBroadcastExchange #2 @@ -35,41 +35,23 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter + CometBroadcastExchange #4 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometBroadcastExchange #5 CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_state] - CometFilter [ca_country,ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + CometBroadcastExchange #6 + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastExchange #7 + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometBroadcastExchange #8 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #9 + CometFilter [r_reason_sk] + CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt index 5bfd2925d2..900d230642 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt @@ -1,25 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (21) -+- * Project (20) - +- Window (19) - +- * Sort (18) - +- Exchange (17) - +- * HashAggregate (16) - +- Exchange (15) - +- * HashAggregate (14) - +- * Expand (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (10) - +- * ColumnarToRow (9) - +- CometFilter (8) - +- CometScan parquet spark_catalog.default.item (7) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * Sort (20) + +- Exchange (19) + +- * HashAggregate (18) + +- Exchange (17) + +- * ColumnarToRow (16) + +- CometHashAggregate (15) + +- CometExpand (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.date_dim (3) + +- CometBroadcastExchange (11) + +- CometFilter (10) + +- CometScan parquet spark_catalog.default.item (9) (1) Scan parquet spark_catalog.default.web_sales @@ -34,122 +36,131 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] +Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] -(7) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] +(9) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#7, i_class#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) - -(9) ColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#7, i_category#8] +(10) CometFilter +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Condition : isnotnull(i_item_sk#7) -(10) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Arguments: [i_item_sk#7, i_class#8, i_category#9] -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#1, ws_net_paid#2] +Right output [3]: [i_item_sk#7, i_class#8, i_category#9] +Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight -(12) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_category#8, i_class#7] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#7, i_category#8] +(13) CometProject +Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#8, i_category#9] +Arguments: [ws_net_paid#2, i_category#9, i_class#8], [ws_net_paid#2, i_category#9, i_class#8] -(13) Expand [codegen id : 3] -Input [3]: [ws_net_paid#2, i_category#8, i_class#7] -Arguments: [[ws_net_paid#2, i_category#8, i_class#7, 0], [ws_net_paid#2, i_category#8, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#9, i_class#10, spark_grouping_id#11] +(14) CometExpand +Input [3]: [ws_net_paid#2, i_category#9, i_class#8] +Arguments: [[ws_net_paid#2, i_category#9, i_class#8, 0], [ws_net_paid#2, i_category#9, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#10, i_class#11, spark_grouping_id#12] -(14) HashAggregate [codegen id : 3] -Input [4]: [ws_net_paid#2, i_category#9, i_class#10, spark_grouping_id#11] -Keys [3]: [i_category#9, i_class#10, spark_grouping_id#11] +(15) CometHashAggregate +Input [4]: [ws_net_paid#2, i_category#10, i_class#11, spark_grouping_id#12] +Keys [3]: [i_category#10, i_class#11, spark_grouping_id#12] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#12] -Results [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] -(15) Exchange -Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] -Arguments: hashpartitioning(i_category#9, i_class#10, spark_grouping_id#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) ColumnarToRow [codegen id : 1] +Input [4]: [i_category#10, i_class#11, spark_grouping_id#12, sum#13] -(16) HashAggregate [codegen id : 4] -Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] -Keys [3]: [i_category#9, i_class#10, spark_grouping_id#11] +(17) Exchange +Input [4]: [i_category#10, i_class#11, spark_grouping_id#12, sum#13] +Arguments: hashpartitioning(i_category#10, i_class#11, spark_grouping_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(18) HashAggregate [codegen id : 2] +Input [4]: [i_category#10, i_class#11, spark_grouping_id#12, sum#13] +Keys [3]: [i_category#10, i_class#11, spark_grouping_id#12] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#14] -Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS total_sum#15, i_category#9, i_class#10, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS lochierarchy#16, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS _w0#17, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS _w1#18, CASE WHEN (cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint) = 0) THEN i_category#9 END AS _w2#19] +Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS total_sum#15, i_category#10, i_class#11, (cast((shiftright(spark_grouping_id#12, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#12, 0) & 1) as tinyint)) AS lochierarchy#16, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS _w0#17, (cast((shiftright(spark_grouping_id#12, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#12, 0) & 1) as tinyint)) AS _w1#18, CASE WHEN (cast((shiftright(spark_grouping_id#12, 0) & 1) as tinyint) = 0) THEN i_category#10 END AS _w2#19] -(17) Exchange -Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] -Arguments: hashpartitioning(_w1#18, _w2#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(19) Exchange +Input [7]: [total_sum#15, i_category#10, i_class#11, lochierarchy#16, _w0#17, _w1#18, _w2#19] +Arguments: hashpartitioning(_w1#18, _w2#19, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(18) Sort [codegen id : 5] -Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] +(20) Sort [codegen id : 3] +Input [7]: [total_sum#15, i_category#10, i_class#11, lochierarchy#16, _w0#17, _w1#18, _w2#19] Arguments: [_w1#18 ASC NULLS FIRST, _w2#19 ASC NULLS FIRST, _w0#17 DESC NULLS LAST], false, 0 -(19) Window -Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] +(21) Window +Input [7]: [total_sum#15, i_category#10, i_class#11, lochierarchy#16, _w0#17, _w1#18, _w2#19] Arguments: [rank(_w0#17) windowspecdefinition(_w1#18, _w2#19, _w0#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#20], [_w1#18, _w2#19], [_w0#17 DESC NULLS LAST] -(20) Project [codegen id : 6] -Output [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] -Input [8]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19, rank_within_parent#20] +(22) Project [codegen id : 4] +Output [5]: [total_sum#15, i_category#10, i_class#11, lochierarchy#16, rank_within_parent#20] +Input [8]: [total_sum#15, i_category#10, i_class#11, lochierarchy#16, _w0#17, _w1#18, _w2#19, rank_within_parent#20] -(21) TakeOrderedAndProject -Input [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] -Arguments: 100, [lochierarchy#16 DESC NULLS LAST, CASE WHEN (lochierarchy#16 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#20 ASC NULLS FIRST], [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] +(23) TakeOrderedAndProject +Input [5]: [total_sum#15, i_category#10, i_class#11, lochierarchy#16, rank_within_parent#20] +Arguments: 100, [lochierarchy#16 DESC NULLS LAST, CASE WHEN (lochierarchy#16 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#20 ASC NULLS FIRST], [total_sum#15, i_category#10, i_class#11, lochierarchy#16, rank_within_parent#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (26) -+- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) -(22) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#21] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(23) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#21] -Condition : (((isnotnull(d_month_seq#21) AND (d_month_seq#21 >= 1200)) AND (d_month_seq#21 <= 1211)) AND isnotnull(d_date_sk#5)) +(25) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(24) CometProject -Input [2]: [d_date_sk#5, d_month_seq#21] +(26) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(25) ColumnarToRow [codegen id : 1] +(27) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(26) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index f9db2ce7a4..4218938c1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (6) + WholeStageCodegen (4) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [_w1,_w2,_w0] InputAdapter Exchange [_w1,_w2] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] InputAdapter Exchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [ws_net_paid,i_category,i_class] - Project [ws_net_paid,i_category,i_class] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -30,12 +30,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #5 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index a4c9f13ce6..0434066e67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -1,51 +1,53 @@ == Physical Plan == -* HashAggregate (47) -+- Exchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin LeftAnti BuildRight (43) - :- * BroadcastHashJoin LeftAnti BuildRight (29) - : :- * HashAggregate (15) - : : +- Exchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.customer (7) - : +- BroadcastExchange (28) - : +- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * ColumnarToRow (18) - : : : +- CometFilter (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - +- BroadcastExchange (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * ColumnarToRow (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.web_sales (30) - : +- ReusedExchange (33) - +- ReusedExchange (36) +* HashAggregate (49) ++- Exchange (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftAnti BuildRight (45) + :- * BroadcastHashJoin LeftAnti BuildRight (31) + : :- * HashAggregate (17) + : : +- Exchange (16) + : : +- * ColumnarToRow (15) + : : +- CometHashAggregate (14) + : : +- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.customer (9) + : +- BroadcastExchange (30) + : +- * HashAggregate (29) + : +- Exchange (28) + : +- * ColumnarToRow (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (44) + +- * HashAggregate (43) + +- Exchange (42) + +- * ColumnarToRow (41) + +- CometHashAggregate (40) + +- CometProject (39) + +- CometBroadcastHashJoin (38) + :- CometProject (36) + : +- CometBroadcastHashJoin (35) + : :- CometFilter (33) + : : +- CometScan parquet spark_catalog.default.web_sales (32) + : +- ReusedExchange (34) + +- ReusedExchange (37) (1) Scan parquet spark_catalog.default.store_sales @@ -60,262 +62,263 @@ ReadSchema: struct Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +(3) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#4, d_date#5] +(4) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#2] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] + +(6) CometBroadcastExchange +Input [2]: [d_date_sk#4, d_date#5] +Arguments: [d_date_sk#4, d_date#5] -(6) Project [codegen id : 3] -Output [2]: [ss_customer_sk#1, d_date#5] +(7) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +Right output [2]: [d_date_sk#4, d_date#5] +Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight + +(8) CometProject Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] +Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] -(7) Scan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +(9) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Condition : isnotnull(c_customer_sk#6) - -(9) ColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +(10) CometFilter +Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Condition : isnotnull(c_customer_sk#7) -(10) BroadcastExchange -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [c_customer_sk#7, c_first_name#8, c_last_name#9] -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#6] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#1, d_date#5] +Right output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight -(12) Project [codegen id : 3] -Output [3]: [c_last_name#8, c_first_name#7, d_date#5] -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#7, c_last_name#8] +(13) CometProject +Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#8, c_last_name#9] +Arguments: [c_last_name#9, c_first_name#8, d_date#5], [c_last_name#9, c_first_name#8, d_date#5] -(13) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +(14) CometHashAggregate +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] -(14) Exchange -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(15) ColumnarToRow [codegen id : 1] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] + +(16) Exchange +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Arguments: hashpartitioning(c_last_name#9, c_first_name#8, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 12] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] -Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +(17) HashAggregate [codegen id : 6] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] +Keys [3]: [c_last_name#9, c_first_name#8, d_date#5] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#8, c_first_name#7, d_date#5] +Results [3]: [c_last_name#9, c_first_name#8, d_date#5] -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +(18) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(17) CometFilter -Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] -Condition : isnotnull(cs_bill_customer_sk#9) - -(18) ColumnarToRow [codegen id : 6] -Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +(19) CometFilter +Input [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] +Condition : isnotnull(cs_bill_customer_sk#10) -(19) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#12, d_date#13] +(20) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#13, d_date#14] -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None +(21) CometBroadcastHashJoin +Left output [2]: [cs_bill_customer_sk#10, cs_sold_date_sk#11] +Right output [2]: [d_date_sk#13, d_date#14] +Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(21) Project [codegen id : 6] -Output [2]: [cs_bill_customer_sk#9, d_date#13] -Input [4]: [cs_bill_customer_sk#9, cs_sold_date_sk#10, d_date_sk#12, d_date#13] +(22) CometProject +Input [4]: [cs_bill_customer_sk#10, cs_sold_date_sk#11, d_date_sk#13, d_date#14] +Arguments: [cs_bill_customer_sk#10, d_date#14], [cs_bill_customer_sk#10, d_date#14] -(22) ReusedExchange [Reuses operator id: 10] -Output [3]: [c_customer_sk#14, c_first_name#15, c_last_name#16] +(23) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_bill_customer_sk#9] -Right keys [1]: [c_customer_sk#14] -Join type: Inner -Join condition: None +(24) CometBroadcastHashJoin +Left output [2]: [cs_bill_customer_sk#10, d_date#14] +Right output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [cs_bill_customer_sk#10], [c_customer_sk#15], Inner, BuildRight -(24) Project [codegen id : 6] -Output [3]: [c_last_name#16, c_first_name#15, d_date#13] -Input [5]: [cs_bill_customer_sk#9, d_date#13, c_customer_sk#14, c_first_name#15, c_last_name#16] +(25) CometProject +Input [5]: [cs_bill_customer_sk#10, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] +Arguments: [c_last_name#17, c_first_name#16, d_date#14], [c_last_name#17, c_first_name#16, d_date#14] -(25) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +(26) CometHashAggregate +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#16, c_first_name#15, d_date#13] -(26) Exchange -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(27) ColumnarToRow [codegen id : 2] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -(27) HashAggregate [codegen id : 7] -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +(28) Exchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(29) HashAggregate [codegen id : 3] +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#16, c_first_name#15, d_date#13] +Results [3]: [c_last_name#17, c_first_name#16, d_date#14] -(28) BroadcastExchange -Input [3]: [c_last_name#16, c_first_name#15, d_date#13] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] +(30) BroadcastExchange +Input [3]: [c_last_name#17, c_first_name#16, d_date#14] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] -(29) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#16, ), isnull(c_last_name#16), coalesce(c_first_name#15, ), isnull(c_first_name#15), coalesce(d_date#13, 1970-01-01), isnull(d_date#13)] +(31) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] Join type: LeftAnti Join condition: None -(30) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +(32) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#18), dynamicpruningexpression(ws_sold_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] -Condition : isnotnull(ws_bill_customer_sk#17) - -(32) ColumnarToRow [codegen id : 10] -Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +(33) CometFilter +Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Condition : isnotnull(ws_bill_customer_sk#18) -(33) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#20, d_date#21] +(34) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#21, d_date#22] -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None +(35) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] +Right output [2]: [d_date_sk#21, d_date#22] +Arguments: [ws_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight -(35) Project [codegen id : 10] -Output [2]: [ws_bill_customer_sk#17, d_date#21] -Input [4]: [ws_bill_customer_sk#17, ws_sold_date_sk#18, d_date_sk#20, d_date#21] +(36) CometProject +Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#21, d_date#22] +Arguments: [ws_bill_customer_sk#18, d_date#22], [ws_bill_customer_sk#18, d_date#22] -(36) ReusedExchange [Reuses operator id: 10] -Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] +(37) ReusedExchange [Reuses operator id: 11] +Output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_bill_customer_sk#17] -Right keys [1]: [c_customer_sk#22] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#18, d_date#22] +Right output [3]: [c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [ws_bill_customer_sk#18], [c_customer_sk#23], Inner, BuildRight -(38) Project [codegen id : 10] -Output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Input [5]: [ws_bill_customer_sk#17, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] +(39) CometProject +Input [5]: [ws_bill_customer_sk#18, d_date#22, c_customer_sk#23, c_first_name#24, c_last_name#25] +Arguments: [c_last_name#25, c_first_name#24, d_date#22], [c_last_name#25, c_first_name#24, d_date#22] -(39) HashAggregate [codegen id : 10] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +(40) CometHashAggregate +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] -(40) Exchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(41) ColumnarToRow [codegen id : 4] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] + +(42) Exchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: hashpartitioning(c_last_name#25, c_first_name#24, d_date#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(41) HashAggregate [codegen id : 11] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +(43) HashAggregate [codegen id : 5] +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Keys [3]: [c_last_name#25, c_first_name#24, d_date#22] Functions: [] Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] +Results [3]: [c_last_name#25, c_first_name#24, d_date#22] -(42) BroadcastExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] +(44) BroadcastExchange +Input [3]: [c_last_name#25, c_first_name#24, d_date#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] +(45) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(c_last_name#9, ), isnull(c_last_name#9), coalesce(c_first_name#8, ), isnull(c_first_name#8), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#25, ), isnull(c_last_name#25), coalesce(c_first_name#24, ), isnull(c_first_name#24), coalesce(d_date#22, 1970-01-01), isnull(d_date#22)] Join type: LeftAnti Join condition: None -(44) Project [codegen id : 12] +(46) Project [codegen id : 6] Output: [] -Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Input [3]: [c_last_name#9, c_first_name#8, d_date#5] -(45) HashAggregate [codegen id : 12] +(47) HashAggregate [codegen id : 6] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#25] -Results [1]: [count#26] +Aggregate Attributes [1]: [count#26] +Results [1]: [count#27] -(46) Exchange -Input [1]: [count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(48) Exchange +Input [1]: [count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(47) HashAggregate [codegen id : 13] -Input [1]: [count#26] +(49) HashAggregate [codegen id : 7] +Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#27] -Results [1]: [count(1)#27 AS count(1)#28] +Aggregate Attributes [1]: [count(1)#28] +Results [1]: [count(1)#28 AS count(1)#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (52) -+- * ColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan parquet spark_catalog.default.date_dim (48) +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) -(48) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +(50) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(49) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) +(51) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) -(50) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +(52) CometProject +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(53) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(52) BroadcastExchange +(54) BroadcastExchange Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 +Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#3 -Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 +Subquery:3 Hosting operator id = 32 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index 315afe6602..cfac83844a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (13) +WholeStageCodegen (7) HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 - WholeStageCodegen (12) + WholeStageCodegen (6) HashAggregate [count,count] Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] @@ -10,14 +10,14 @@ WholeStageCodegen (13) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter Exchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ss_customer_sk,c_customer_sk] + CometProject [ss_customer_sk,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -28,54 +28,48 @@ WholeStageCodegen (13) CometProject [d_date_sk,d_date] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange #4 + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange #5 + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (3) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [c_last_name,c_first_name,d_date] #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,d_date] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometFilter [cs_bill_customer_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) + BroadcastExchange #8 + WholeStageCodegen (5) HashAggregate [c_last_name,c_first_name,d_date] InputAdapter - Exchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [c_last_name,c_first_name,d_date] #9 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometProject [c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + CometProject [ws_bill_customer_sk,d_date] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_bill_customer_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #4 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index a5f68e5648..3f905ebe90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt @@ -1,186 +1,176 @@ == Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (182) -:- * BroadcastNestedLoopJoin Inner BuildRight (160) -: :- * BroadcastNestedLoopJoin Inner BuildRight (138) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (116) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (94) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (72) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : : : : : :- * HashAggregate (28) -: : : : : : : +- Exchange (27) -: : : : : : : +- * HashAggregate (26) -: : : : : : : +- * Project (25) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (24) -: : : : : : : :- * Project (18) -: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) -: : : : : : : : :- * Project (11) -: : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (10) -: : : : : : : : : :- * ColumnarToRow (4) -: : : : : : : : : : +- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- BroadcastExchange (9) -: : : : : : : : : +- * ColumnarToRow (8) -: : : : : : : : : +- CometProject (7) -: : : : : : : : : +- CometFilter (6) -: : : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (5) -: : : : : : : : +- BroadcastExchange (16) -: : : : : : : : +- * ColumnarToRow (15) -: : : : : : : : +- CometProject (14) -: : : : : : : : +- CometFilter (13) -: : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (12) -: : : : : : : +- BroadcastExchange (23) -: : : : : : : +- * ColumnarToRow (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometFilter (20) -: : : : : : : +- CometScan parquet spark_catalog.default.store (19) -: : : : : : +- BroadcastExchange (49) -: : : : : : +- * HashAggregate (48) -: : : : : : +- Exchange (47) -: : : : : : +- * HashAggregate (46) -: : : : : : +- * Project (45) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (44) -: : : : : : :- * Project (42) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) -: : : : : : : :- * Project (35) -: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) -: : : : : : : : :- * ColumnarToRow (32) -: : : : : : : : : +- CometProject (31) -: : : : : : : : : +- CometFilter (30) -: : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) -: : : : : : : : +- ReusedExchange (33) -: : : : : : : +- BroadcastExchange (40) -: : : : : : : +- * ColumnarToRow (39) -: : : : : : : +- CometProject (38) -: : : : : : : +- CometFilter (37) -: : : : : : : +- CometScan parquet spark_catalog.default.time_dim (36) -: : : : : : +- ReusedExchange (43) -: : : : : +- BroadcastExchange (71) -: : : : : +- * HashAggregate (70) -: : : : : +- Exchange (69) -: : : : : +- * HashAggregate (68) -: : : : : +- * Project (67) -: : : : : +- * BroadcastHashJoin Inner BuildRight (66) -: : : : : :- * Project (64) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (63) -: : : : : : :- * Project (57) -: : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) -: : : : : : : :- * ColumnarToRow (54) -: : : : : : : : +- CometProject (53) -: : : : : : : : +- CometFilter (52) -: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (51) -: : : : : : : +- ReusedExchange (55) -: : : : : : +- BroadcastExchange (62) -: : : : : : +- * ColumnarToRow (61) -: : : : : : +- CometProject (60) -: : : : : : +- CometFilter (59) -: : : : : : +- CometScan parquet spark_catalog.default.time_dim (58) -: : : : : +- ReusedExchange (65) -: : : : +- BroadcastExchange (93) -: : : : +- * HashAggregate (92) -: : : : +- Exchange (91) -: : : : +- * HashAggregate (90) -: : : : +- * Project (89) -: : : : +- * BroadcastHashJoin Inner BuildRight (88) -: : : : :- * Project (86) -: : : : : +- * BroadcastHashJoin Inner BuildRight (85) -: : : : : :- * Project (79) -: : : : : : +- * BroadcastHashJoin Inner BuildRight (78) -: : : : : : :- * ColumnarToRow (76) -: : : : : : : +- CometProject (75) -: : : : : : : +- CometFilter (74) -: : : : : : : +- CometScan parquet spark_catalog.default.store_sales (73) -: : : : : : +- ReusedExchange (77) -: : : : : +- BroadcastExchange (84) -: : : : : +- * ColumnarToRow (83) -: : : : : +- CometProject (82) -: : : : : +- CometFilter (81) -: : : : : +- CometScan parquet spark_catalog.default.time_dim (80) -: : : : +- ReusedExchange (87) -: : : +- BroadcastExchange (115) -: : : +- * HashAggregate (114) -: : : +- Exchange (113) -: : : +- * HashAggregate (112) -: : : +- * Project (111) -: : : +- * BroadcastHashJoin Inner BuildRight (110) -: : : :- * Project (108) -: : : : +- * BroadcastHashJoin Inner BuildRight (107) -: : : : :- * Project (101) -: : : : : +- * BroadcastHashJoin Inner BuildRight (100) -: : : : : :- * ColumnarToRow (98) -: : : : : : +- CometProject (97) -: : : : : : +- CometFilter (96) -: : : : : : +- CometScan parquet spark_catalog.default.store_sales (95) -: : : : : +- ReusedExchange (99) -: : : : +- BroadcastExchange (106) -: : : : +- * ColumnarToRow (105) -: : : : +- CometProject (104) -: : : : +- CometFilter (103) -: : : : +- CometScan parquet spark_catalog.default.time_dim (102) -: : : +- ReusedExchange (109) -: : +- BroadcastExchange (137) -: : +- * HashAggregate (136) -: : +- Exchange (135) -: : +- * HashAggregate (134) -: : +- * Project (133) -: : +- * BroadcastHashJoin Inner BuildRight (132) -: : :- * Project (130) -: : : +- * BroadcastHashJoin Inner BuildRight (129) -: : : :- * Project (123) -: : : : +- * BroadcastHashJoin Inner BuildRight (122) -: : : : :- * ColumnarToRow (120) -: : : : : +- CometProject (119) -: : : : : +- CometFilter (118) -: : : : : +- CometScan parquet spark_catalog.default.store_sales (117) -: : : : +- ReusedExchange (121) -: : : +- BroadcastExchange (128) -: : : +- * ColumnarToRow (127) -: : : +- CometProject (126) -: : : +- CometFilter (125) -: : : +- CometScan parquet spark_catalog.default.time_dim (124) -: : +- ReusedExchange (131) -: +- BroadcastExchange (159) -: +- * HashAggregate (158) -: +- Exchange (157) -: +- * HashAggregate (156) -: +- * Project (155) -: +- * BroadcastHashJoin Inner BuildRight (154) -: :- * Project (152) -: : +- * BroadcastHashJoin Inner BuildRight (151) -: : :- * Project (145) -: : : +- * BroadcastHashJoin Inner BuildRight (144) -: : : :- * ColumnarToRow (142) -: : : : +- CometProject (141) -: : : : +- CometFilter (140) -: : : : +- CometScan parquet spark_catalog.default.store_sales (139) -: : : +- ReusedExchange (143) -: : +- BroadcastExchange (150) -: : +- * ColumnarToRow (149) -: : +- CometProject (148) -: : +- CometFilter (147) -: : +- CometScan parquet spark_catalog.default.time_dim (146) -: +- ReusedExchange (153) -+- BroadcastExchange (181) - +- * HashAggregate (180) - +- Exchange (179) - +- * HashAggregate (178) - +- * Project (177) - +- * BroadcastHashJoin Inner BuildRight (176) - :- * Project (174) - : +- * BroadcastHashJoin Inner BuildRight (173) - : :- * Project (167) - : : +- * BroadcastHashJoin Inner BuildRight (166) - : : :- * ColumnarToRow (164) - : : : +- CometProject (163) - : : : +- CometFilter (162) - : : : +- CometScan parquet spark_catalog.default.store_sales (161) - : : +- ReusedExchange (165) - : +- BroadcastExchange (172) - : +- * ColumnarToRow (171) - : +- CometProject (170) - : +- CometFilter (169) - : +- CometScan parquet spark_catalog.default.time_dim (168) - +- ReusedExchange (175) +* BroadcastNestedLoopJoin Inner BuildRight (172) +:- * BroadcastNestedLoopJoin Inner BuildRight (151) +: :- * BroadcastNestedLoopJoin Inner BuildRight (130) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) +: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) +: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) +: : : : : : :- * HashAggregate (25) +: : : : : : : +- Exchange (24) +: : : : : : : +- * ColumnarToRow (23) +: : : : : : : +- CometHashAggregate (22) +: : : : : : : +- CometProject (21) +: : : : : : : +- CometBroadcastHashJoin (20) +: : : : : : : :- CometProject (15) +: : : : : : : : +- CometBroadcastHashJoin (14) +: : : : : : : : :- CometProject (9) +: : : : : : : : : +- CometBroadcastHashJoin (8) +: : : : : : : : : :- CometProject (3) +: : : : : : : : : : +- CometFilter (2) +: : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : : : : : : +- CometBroadcastExchange (7) +: : : : : : : : : +- CometProject (6) +: : : : : : : : : +- CometFilter (5) +: : : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (4) +: : : : : : : : +- CometBroadcastExchange (13) +: : : : : : : : +- CometProject (12) +: : : : : : : : +- CometFilter (11) +: : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (10) +: : : : : : : +- CometBroadcastExchange (19) +: : : : : : : +- CometProject (18) +: : : : : : : +- CometFilter (17) +: : : : : : : +- CometScan parquet spark_catalog.default.store (16) +: : : : : : +- BroadcastExchange (45) +: : : : : : +- * HashAggregate (44) +: : : : : : +- Exchange (43) +: : : : : : +- * ColumnarToRow (42) +: : : : : : +- CometHashAggregate (41) +: : : : : : +- CometProject (40) +: : : : : : +- CometBroadcastHashJoin (39) +: : : : : : :- CometProject (37) +: : : : : : : +- CometBroadcastHashJoin (36) +: : : : : : : :- CometProject (31) +: : : : : : : : +- CometBroadcastHashJoin (30) +: : : : : : : : :- CometProject (28) +: : : : : : : : : +- CometFilter (27) +: : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (26) +: : : : : : : : +- ReusedExchange (29) +: : : : : : : +- CometBroadcastExchange (35) +: : : : : : : +- CometProject (34) +: : : : : : : +- CometFilter (33) +: : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) +: : : : : : +- ReusedExchange (38) +: : : : : +- BroadcastExchange (66) +: : : : : +- * HashAggregate (65) +: : : : : +- Exchange (64) +: : : : : +- * ColumnarToRow (63) +: : : : : +- CometHashAggregate (62) +: : : : : +- CometProject (61) +: : : : : +- CometBroadcastHashJoin (60) +: : : : : :- CometProject (58) +: : : : : : +- CometBroadcastHashJoin (57) +: : : : : : :- CometProject (52) +: : : : : : : +- CometBroadcastHashJoin (51) +: : : : : : : :- CometProject (49) +: : : : : : : : +- CometFilter (48) +: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (47) +: : : : : : : +- ReusedExchange (50) +: : : : : : +- CometBroadcastExchange (56) +: : : : : : +- CometProject (55) +: : : : : : +- CometFilter (54) +: : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) +: : : : : +- ReusedExchange (59) +: : : : +- BroadcastExchange (87) +: : : : +- * HashAggregate (86) +: : : : +- Exchange (85) +: : : : +- * ColumnarToRow (84) +: : : : +- CometHashAggregate (83) +: : : : +- CometProject (82) +: : : : +- CometBroadcastHashJoin (81) +: : : : :- CometProject (79) +: : : : : +- CometBroadcastHashJoin (78) +: : : : : :- CometProject (73) +: : : : : : +- CometBroadcastHashJoin (72) +: : : : : : :- CometProject (70) +: : : : : : : +- CometFilter (69) +: : : : : : : +- CometScan parquet spark_catalog.default.store_sales (68) +: : : : : : +- ReusedExchange (71) +: : : : : +- CometBroadcastExchange (77) +: : : : : +- CometProject (76) +: : : : : +- CometFilter (75) +: : : : : +- CometScan parquet spark_catalog.default.time_dim (74) +: : : : +- ReusedExchange (80) +: : : +- BroadcastExchange (108) +: : : +- * HashAggregate (107) +: : : +- Exchange (106) +: : : +- * ColumnarToRow (105) +: : : +- CometHashAggregate (104) +: : : +- CometProject (103) +: : : +- CometBroadcastHashJoin (102) +: : : :- CometProject (100) +: : : : +- CometBroadcastHashJoin (99) +: : : : :- CometProject (94) +: : : : : +- CometBroadcastHashJoin (93) +: : : : : :- CometProject (91) +: : : : : : +- CometFilter (90) +: : : : : : +- CometScan parquet spark_catalog.default.store_sales (89) +: : : : : +- ReusedExchange (92) +: : : : +- CometBroadcastExchange (98) +: : : : +- CometProject (97) +: : : : +- CometFilter (96) +: : : : +- CometScan parquet spark_catalog.default.time_dim (95) +: : : +- ReusedExchange (101) +: : +- BroadcastExchange (129) +: : +- * HashAggregate (128) +: : +- Exchange (127) +: : +- * ColumnarToRow (126) +: : +- CometHashAggregate (125) +: : +- CometProject (124) +: : +- CometBroadcastHashJoin (123) +: : :- CometProject (121) +: : : +- CometBroadcastHashJoin (120) +: : : :- CometProject (115) +: : : : +- CometBroadcastHashJoin (114) +: : : : :- CometProject (112) +: : : : : +- CometFilter (111) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (110) +: : : : +- ReusedExchange (113) +: : : +- CometBroadcastExchange (119) +: : : +- CometProject (118) +: : : +- CometFilter (117) +: : : +- CometScan parquet spark_catalog.default.time_dim (116) +: : +- ReusedExchange (122) +: +- BroadcastExchange (150) +: +- * HashAggregate (149) +: +- Exchange (148) +: +- * ColumnarToRow (147) +: +- CometHashAggregate (146) +: +- CometProject (145) +: +- CometBroadcastHashJoin (144) +: :- CometProject (142) +: : +- CometBroadcastHashJoin (141) +: : :- CometProject (136) +: : : +- CometBroadcastHashJoin (135) +: : : :- CometProject (133) +: : : : +- CometFilter (132) +: : : : +- CometScan parquet spark_catalog.default.store_sales (131) +: : : +- ReusedExchange (134) +: : +- CometBroadcastExchange (140) +: : +- CometProject (139) +: : +- CometFilter (138) +: : +- CometScan parquet spark_catalog.default.time_dim (137) +: +- ReusedExchange (143) ++- BroadcastExchange (171) + +- * HashAggregate (170) + +- Exchange (169) + +- * ColumnarToRow (168) + +- CometHashAggregate (167) + +- CometProject (166) + +- CometBroadcastHashJoin (165) + :- CometProject (163) + : +- CometBroadcastHashJoin (162) + : :- CometProject (157) + : : +- CometBroadcastHashJoin (156) + : : :- CometProject (154) + : : : +- CometFilter (153) + : : : +- CometScan parquet spark_catalog.default.store_sales (152) + : : +- ReusedExchange (155) + : +- CometBroadcastExchange (161) + : +- CometProject (160) + : +- CometFilter (159) + : +- CometScan parquet spark_catalog.default.time_dim (158) + +- ReusedExchange (164) (1) Scan parquet spark_catalog.default.store_sales @@ -198,834 +188,756 @@ Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isn Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -(4) ColumnarToRow [codegen id : 4] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(5) Scan parquet spark_catalog.default.household_demographics +(4) Scan parquet spark_catalog.default.household_demographics Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) -(7) CometProject +(6) CometProject Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] Arguments: [hd_demo_sk#5], [hd_demo_sk#5] -(8) ColumnarToRow [codegen id : 1] +(7) CometBroadcastExchange Input [1]: [hd_demo_sk#5] +Arguments: [hd_demo_sk#5] -(9) BroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(8) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Right output [1]: [hd_demo_sk#5] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight -(10) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 4] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +(9) CometProject Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] -(12) Scan parquet spark_catalog.default.time_dim +(10) Scan parquet spark_catalog.default.time_dim Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(13) CometFilter +(11) CometFilter Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) -(14) CometProject +(12) CometProject Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] Arguments: [t_time_sk#8], [t_time_sk#8] -(15) ColumnarToRow [codegen id : 2] +(13) CometBroadcastExchange Input [1]: [t_time_sk#8] +Arguments: [t_time_sk#8] -(16) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(14) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Right output [1]: [t_time_sk#8] +Arguments: [ss_sold_time_sk#1], [t_time_sk#8], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [1]: [ss_store_sk#3] +(15) CometProject Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] +Arguments: [ss_store_sk#3], [ss_store_sk#3] -(19) Scan parquet spark_catalog.default.store +(16) Scan parquet spark_catalog.default.store Output [2]: [s_store_sk#11, s_store_name#12] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] ReadSchema: struct -(20) CometFilter +(17) CometFilter Input [2]: [s_store_sk#11, s_store_name#12] Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) -(21) CometProject +(18) CometProject Input [2]: [s_store_sk#11, s_store_name#12] Arguments: [s_store_sk#11], [s_store_sk#11] -(22) ColumnarToRow [codegen id : 3] +(19) CometBroadcastExchange Input [1]: [s_store_sk#11] +Arguments: [s_store_sk#11] -(23) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#3] +Right output [1]: [s_store_sk#11] +Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight -(25) Project [codegen id : 4] -Output: [] +(21) CometProject Input [2]: [ss_store_sk#3, s_store_sk#11] -(26) HashAggregate [codegen id : 4] +(22) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#13] -Results [1]: [count#14] -(27) Exchange -Input [1]: [count#14] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(23) ColumnarToRow [codegen id : 1] +Input [1]: [count#13] + +(24) Exchange +Input [1]: [count#13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 40] -Input [1]: [count#14] +(25) HashAggregate [codegen id : 16] +Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#15] -Results [1]: [count(1)#15 AS h8_30_to_9#16] +Aggregate Attributes [1]: [count(1)#14] +Results [1]: [count(1)#14 AS h8_30_to_9#15] -(29) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] +(26) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_hdemo_sk#18) AND isnotnull(ss_sold_time_sk#17)) AND isnotnull(ss_store_sk#19)) +(27) CometFilter +Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] +Condition : ((isnotnull(ss_hdemo_sk#17) AND isnotnull(ss_sold_time_sk#16)) AND isnotnull(ss_store_sk#18)) -(31) CometProject -Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] -Arguments: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19], [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] +(28) CometProject +Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, ss_sold_date_sk#19] +Arguments: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18], [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18] -(32) ColumnarToRow [codegen id : 8] -Input [3]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] +(29) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#20] -(33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#21] +(30) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18] +Right output [1]: [hd_demo_sk#20] +Arguments: [ss_hdemo_sk#17], [hd_demo_sk#20], Inner, BuildRight -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#18] -Right keys [1]: [hd_demo_sk#21] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 8] -Output [2]: [ss_sold_time_sk#17, ss_store_sk#19] -Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, hd_demo_sk#21] +(31) CometProject +Input [4]: [ss_sold_time_sk#16, ss_hdemo_sk#17, ss_store_sk#18, hd_demo_sk#20] +Arguments: [ss_sold_time_sk#16, ss_store_sk#18], [ss_sold_time_sk#16, ss_store_sk#18] -(36) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#22, t_hour#23, t_minute#24] +(32) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#21, t_hour#22, t_minute#23] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(37) CometFilter -Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] -Condition : ((((isnotnull(t_hour#23) AND isnotnull(t_minute#24)) AND (t_hour#23 = 9)) AND (t_minute#24 < 30)) AND isnotnull(t_time_sk#22)) +(33) CometFilter +Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Condition : ((((isnotnull(t_hour#22) AND isnotnull(t_minute#23)) AND (t_hour#22 = 9)) AND (t_minute#23 < 30)) AND isnotnull(t_time_sk#21)) -(38) CometProject -Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] -Arguments: [t_time_sk#22], [t_time_sk#22] +(34) CometProject +Input [3]: [t_time_sk#21, t_hour#22, t_minute#23] +Arguments: [t_time_sk#21], [t_time_sk#21] -(39) ColumnarToRow [codegen id : 6] -Input [1]: [t_time_sk#22] +(35) CometBroadcastExchange +Input [1]: [t_time_sk#21] +Arguments: [t_time_sk#21] -(40) BroadcastExchange -Input [1]: [t_time_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#16, ss_store_sk#18] +Right output [1]: [t_time_sk#21] +Arguments: [ss_sold_time_sk#16], [t_time_sk#21], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#17] -Right keys [1]: [t_time_sk#22] -Join type: Inner -Join condition: None +(37) CometProject +Input [3]: [ss_sold_time_sk#16, ss_store_sk#18, t_time_sk#21] +Arguments: [ss_store_sk#18], [ss_store_sk#18] -(42) Project [codegen id : 8] -Output [1]: [ss_store_sk#19] -Input [3]: [ss_sold_time_sk#17, ss_store_sk#19, t_time_sk#22] +(38) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#24] -(43) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#25] +(39) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#18] +Right output [1]: [s_store_sk#24] +Arguments: [ss_store_sk#18], [s_store_sk#24], Inner, BuildRight -(44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#19] -Right keys [1]: [s_store_sk#25] -Join type: Inner -Join condition: None +(40) CometProject +Input [2]: [ss_store_sk#18, s_store_sk#24] -(45) Project [codegen id : 8] -Output: [] -Input [2]: [ss_store_sk#19, s_store_sk#25] - -(46) HashAggregate [codegen id : 8] +(41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#26] -Results [1]: [count#27] -(47) Exchange -Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(42) ColumnarToRow [codegen id : 2] +Input [1]: [count#25] -(48) HashAggregate [codegen id : 9] -Input [1]: [count#27] +(43) Exchange +Input [1]: [count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] + +(44) HashAggregate [codegen id : 3] +Input [1]: [count#25] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#28] -Results [1]: [count(1)#28 AS h9_to_9_30#29] +Aggregate Attributes [1]: [count(1)#26] +Results [1]: [count(1)#26 AS h9_to_9_30#27] -(49) BroadcastExchange -Input [1]: [h9_to_9_30#29] -Arguments: IdentityBroadcastMode, [plan_id=7] +(45) BroadcastExchange +Input [1]: [h9_to_9_30#27] +Arguments: IdentityBroadcastMode, [plan_id=3] -(50) BroadcastNestedLoopJoin [codegen id : 40] +(46) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(51) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] +(47) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(52) CometFilter -Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_hdemo_sk#31) AND isnotnull(ss_sold_time_sk#30)) AND isnotnull(ss_store_sk#32)) +(48) CometFilter +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] +Condition : ((isnotnull(ss_hdemo_sk#29) AND isnotnull(ss_sold_time_sk#28)) AND isnotnull(ss_store_sk#30)) -(53) CometProject -Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] -Arguments: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32], [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] +(49) CometProject +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, ss_sold_date_sk#31] +Arguments: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30], [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30] -(54) ColumnarToRow [codegen id : 13] -Input [3]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] +(50) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#32] -(55) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#34] - -(56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#31] -Right keys [1]: [hd_demo_sk#34] -Join type: Inner -Join condition: None +(51) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30] +Right output [1]: [hd_demo_sk#32] +Arguments: [ss_hdemo_sk#29], [hd_demo_sk#32], Inner, BuildRight -(57) Project [codegen id : 13] -Output [2]: [ss_sold_time_sk#30, ss_store_sk#32] -Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, hd_demo_sk#34] +(52) CometProject +Input [4]: [ss_sold_time_sk#28, ss_hdemo_sk#29, ss_store_sk#30, hd_demo_sk#32] +Arguments: [ss_sold_time_sk#28, ss_store_sk#30], [ss_sold_time_sk#28, ss_store_sk#30] -(58) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#35, t_hour#36, t_minute#37] +(53) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#33, t_hour#34, t_minute#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(59) CometFilter -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Condition : ((((isnotnull(t_hour#36) AND isnotnull(t_minute#37)) AND (t_hour#36 = 9)) AND (t_minute#37 >= 30)) AND isnotnull(t_time_sk#35)) +(54) CometFilter +Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Condition : ((((isnotnull(t_hour#34) AND isnotnull(t_minute#35)) AND (t_hour#34 = 9)) AND (t_minute#35 >= 30)) AND isnotnull(t_time_sk#33)) -(60) CometProject -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [t_time_sk#35], [t_time_sk#35] +(55) CometProject +Input [3]: [t_time_sk#33, t_hour#34, t_minute#35] +Arguments: [t_time_sk#33], [t_time_sk#33] -(61) ColumnarToRow [codegen id : 11] -Input [1]: [t_time_sk#35] +(56) CometBroadcastExchange +Input [1]: [t_time_sk#33] +Arguments: [t_time_sk#33] -(62) BroadcastExchange -Input [1]: [t_time_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +(57) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#28, ss_store_sk#30] +Right output [1]: [t_time_sk#33] +Arguments: [ss_sold_time_sk#28], [t_time_sk#33], Inner, BuildRight -(63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#30] -Right keys [1]: [t_time_sk#35] -Join type: Inner -Join condition: None - -(64) Project [codegen id : 13] -Output [1]: [ss_store_sk#32] -Input [3]: [ss_sold_time_sk#30, ss_store_sk#32, t_time_sk#35] +(58) CometProject +Input [3]: [ss_sold_time_sk#28, ss_store_sk#30, t_time_sk#33] +Arguments: [ss_store_sk#30], [ss_store_sk#30] -(65) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#38] +(59) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#36] -(66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#32] -Right keys [1]: [s_store_sk#38] -Join type: Inner -Join condition: None +(60) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#30] +Right output [1]: [s_store_sk#36] +Arguments: [ss_store_sk#30], [s_store_sk#36], Inner, BuildRight -(67) Project [codegen id : 13] -Output: [] -Input [2]: [ss_store_sk#32, s_store_sk#38] +(61) CometProject +Input [2]: [ss_store_sk#30, s_store_sk#36] -(68) HashAggregate [codegen id : 13] +(62) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#39] -Results [1]: [count#40] -(69) Exchange -Input [1]: [count#40] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +(63) ColumnarToRow [codegen id : 4] +Input [1]: [count#37] -(70) HashAggregate [codegen id : 14] -Input [1]: [count#40] +(64) Exchange +Input [1]: [count#37] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(65) HashAggregate [codegen id : 5] +Input [1]: [count#37] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#41] -Results [1]: [count(1)#41 AS h9_30_to_10#42] +Aggregate Attributes [1]: [count(1)#38] +Results [1]: [count(1)#38 AS h9_30_to_10#39] -(71) BroadcastExchange -Input [1]: [h9_30_to_10#42] -Arguments: IdentityBroadcastMode, [plan_id=10] +(66) BroadcastExchange +Input [1]: [h9_30_to_10#39] +Arguments: IdentityBroadcastMode, [plan_id=5] -(72) BroadcastNestedLoopJoin [codegen id : 40] +(67) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(73) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] +(68) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(74) CometFilter -Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_hdemo_sk#44) AND isnotnull(ss_sold_time_sk#43)) AND isnotnull(ss_store_sk#45)) - -(75) CometProject -Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] -Arguments: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45], [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] +(69) CometFilter +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] +Condition : ((isnotnull(ss_hdemo_sk#41) AND isnotnull(ss_sold_time_sk#40)) AND isnotnull(ss_store_sk#42)) -(76) ColumnarToRow [codegen id : 18] -Input [3]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] +(70) CometProject +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, ss_sold_date_sk#43] +Arguments: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42], [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42] -(77) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#47] +(71) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#44] -(78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#44] -Right keys [1]: [hd_demo_sk#47] -Join type: Inner -Join condition: None +(72) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42] +Right output [1]: [hd_demo_sk#44] +Arguments: [ss_hdemo_sk#41], [hd_demo_sk#44], Inner, BuildRight -(79) Project [codegen id : 18] -Output [2]: [ss_sold_time_sk#43, ss_store_sk#45] -Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, hd_demo_sk#47] +(73) CometProject +Input [4]: [ss_sold_time_sk#40, ss_hdemo_sk#41, ss_store_sk#42, hd_demo_sk#44] +Arguments: [ss_sold_time_sk#40, ss_store_sk#42], [ss_sold_time_sk#40, ss_store_sk#42] -(80) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#48, t_hour#49, t_minute#50] +(74) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#45, t_hour#46, t_minute#47] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(81) CometFilter -Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] -Condition : ((((isnotnull(t_hour#49) AND isnotnull(t_minute#50)) AND (t_hour#49 = 10)) AND (t_minute#50 < 30)) AND isnotnull(t_time_sk#48)) - -(82) CometProject -Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] -Arguments: [t_time_sk#48], [t_time_sk#48] +(75) CometFilter +Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Condition : ((((isnotnull(t_hour#46) AND isnotnull(t_minute#47)) AND (t_hour#46 = 10)) AND (t_minute#47 < 30)) AND isnotnull(t_time_sk#45)) -(83) ColumnarToRow [codegen id : 16] -Input [1]: [t_time_sk#48] +(76) CometProject +Input [3]: [t_time_sk#45, t_hour#46, t_minute#47] +Arguments: [t_time_sk#45], [t_time_sk#45] -(84) BroadcastExchange -Input [1]: [t_time_sk#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +(77) CometBroadcastExchange +Input [1]: [t_time_sk#45] +Arguments: [t_time_sk#45] -(85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#43] -Right keys [1]: [t_time_sk#48] -Join type: Inner -Join condition: None +(78) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#40, ss_store_sk#42] +Right output [1]: [t_time_sk#45] +Arguments: [ss_sold_time_sk#40], [t_time_sk#45], Inner, BuildRight -(86) Project [codegen id : 18] -Output [1]: [ss_store_sk#45] -Input [3]: [ss_sold_time_sk#43, ss_store_sk#45, t_time_sk#48] +(79) CometProject +Input [3]: [ss_sold_time_sk#40, ss_store_sk#42, t_time_sk#45] +Arguments: [ss_store_sk#42], [ss_store_sk#42] -(87) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#51] +(80) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#48] -(88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#45] -Right keys [1]: [s_store_sk#51] -Join type: Inner -Join condition: None +(81) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#42] +Right output [1]: [s_store_sk#48] +Arguments: [ss_store_sk#42], [s_store_sk#48], Inner, BuildRight -(89) Project [codegen id : 18] -Output: [] -Input [2]: [ss_store_sk#45, s_store_sk#51] +(82) CometProject +Input [2]: [ss_store_sk#42, s_store_sk#48] -(90) HashAggregate [codegen id : 18] +(83) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#52] -Results [1]: [count#53] -(91) Exchange -Input [1]: [count#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(84) ColumnarToRow [codegen id : 6] +Input [1]: [count#49] -(92) HashAggregate [codegen id : 19] -Input [1]: [count#53] +(85) Exchange +Input [1]: [count#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] + +(86) HashAggregate [codegen id : 7] +Input [1]: [count#49] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#54] -Results [1]: [count(1)#54 AS h10_to_10_30#55] +Aggregate Attributes [1]: [count(1)#50] +Results [1]: [count(1)#50 AS h10_to_10_30#51] -(93) BroadcastExchange -Input [1]: [h10_to_10_30#55] -Arguments: IdentityBroadcastMode, [plan_id=13] +(87) BroadcastExchange +Input [1]: [h10_to_10_30#51] +Arguments: IdentityBroadcastMode, [plan_id=7] -(94) BroadcastNestedLoopJoin [codegen id : 40] +(88) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(95) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] +(89) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(96) CometFilter -Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_hdemo_sk#57) AND isnotnull(ss_sold_time_sk#56)) AND isnotnull(ss_store_sk#58)) +(90) CometFilter +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] +Condition : ((isnotnull(ss_hdemo_sk#53) AND isnotnull(ss_sold_time_sk#52)) AND isnotnull(ss_store_sk#54)) -(97) CometProject -Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] -Arguments: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58], [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] +(91) CometProject +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, ss_sold_date_sk#55] +Arguments: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54], [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54] -(98) ColumnarToRow [codegen id : 23] -Input [3]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] +(92) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#56] -(99) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#60] +(93) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54] +Right output [1]: [hd_demo_sk#56] +Arguments: [ss_hdemo_sk#53], [hd_demo_sk#56], Inner, BuildRight -(100) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#57] -Right keys [1]: [hd_demo_sk#60] -Join type: Inner -Join condition: None - -(101) Project [codegen id : 23] -Output [2]: [ss_sold_time_sk#56, ss_store_sk#58] -Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, hd_demo_sk#60] +(94) CometProject +Input [4]: [ss_sold_time_sk#52, ss_hdemo_sk#53, ss_store_sk#54, hd_demo_sk#56] +Arguments: [ss_sold_time_sk#52, ss_store_sk#54], [ss_sold_time_sk#52, ss_store_sk#54] -(102) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#61, t_hour#62, t_minute#63] +(95) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(103) CometFilter -Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] -Condition : ((((isnotnull(t_hour#62) AND isnotnull(t_minute#63)) AND (t_hour#62 = 10)) AND (t_minute#63 >= 30)) AND isnotnull(t_time_sk#61)) +(96) CometFilter +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Condition : ((((isnotnull(t_hour#58) AND isnotnull(t_minute#59)) AND (t_hour#58 = 10)) AND (t_minute#59 >= 30)) AND isnotnull(t_time_sk#57)) -(104) CometProject -Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] -Arguments: [t_time_sk#61], [t_time_sk#61] +(97) CometProject +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Arguments: [t_time_sk#57], [t_time_sk#57] -(105) ColumnarToRow [codegen id : 21] -Input [1]: [t_time_sk#61] +(98) CometBroadcastExchange +Input [1]: [t_time_sk#57] +Arguments: [t_time_sk#57] -(106) BroadcastExchange -Input [1]: [t_time_sk#61] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(99) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#52, ss_store_sk#54] +Right output [1]: [t_time_sk#57] +Arguments: [ss_sold_time_sk#52], [t_time_sk#57], Inner, BuildRight -(107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#56] -Right keys [1]: [t_time_sk#61] -Join type: Inner -Join condition: None +(100) CometProject +Input [3]: [ss_sold_time_sk#52, ss_store_sk#54, t_time_sk#57] +Arguments: [ss_store_sk#54], [ss_store_sk#54] -(108) Project [codegen id : 23] -Output [1]: [ss_store_sk#58] -Input [3]: [ss_sold_time_sk#56, ss_store_sk#58, t_time_sk#61] +(101) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#60] -(109) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#64] +(102) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#54] +Right output [1]: [s_store_sk#60] +Arguments: [ss_store_sk#54], [s_store_sk#60], Inner, BuildRight -(110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#58] -Right keys [1]: [s_store_sk#64] -Join type: Inner -Join condition: None - -(111) Project [codegen id : 23] -Output: [] -Input [2]: [ss_store_sk#58, s_store_sk#64] +(103) CometProject +Input [2]: [ss_store_sk#54, s_store_sk#60] -(112) HashAggregate [codegen id : 23] +(104) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#65] -Results [1]: [count#66] -(113) Exchange -Input [1]: [count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] +(105) ColumnarToRow [codegen id : 8] +Input [1]: [count#61] -(114) HashAggregate [codegen id : 24] -Input [1]: [count#66] +(106) Exchange +Input [1]: [count#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] + +(107) HashAggregate [codegen id : 9] +Input [1]: [count#61] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#67] -Results [1]: [count(1)#67 AS h10_30_to_11#68] +Aggregate Attributes [1]: [count(1)#62] +Results [1]: [count(1)#62 AS h10_30_to_11#63] -(115) BroadcastExchange -Input [1]: [h10_30_to_11#68] -Arguments: IdentityBroadcastMode, [plan_id=16] +(108) BroadcastExchange +Input [1]: [h10_30_to_11#63] +Arguments: IdentityBroadcastMode, [plan_id=9] -(116) BroadcastNestedLoopJoin [codegen id : 40] +(109) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(117) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +(110) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(118) CometFilter -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) - -(119) CometProject -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] -Arguments: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71], [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +(111) CometFilter +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] +Condition : ((isnotnull(ss_hdemo_sk#65) AND isnotnull(ss_sold_time_sk#64)) AND isnotnull(ss_store_sk#66)) -(120) ColumnarToRow [codegen id : 28] -Input [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +(112) CometProject +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, ss_sold_date_sk#67] +Arguments: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66], [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66] -(121) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#73] +(113) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#68] -(122) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#70] -Right keys [1]: [hd_demo_sk#73] -Join type: Inner -Join condition: None +(114) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66] +Right output [1]: [hd_demo_sk#68] +Arguments: [ss_hdemo_sk#65], [hd_demo_sk#68], Inner, BuildRight -(123) Project [codegen id : 28] -Output [2]: [ss_sold_time_sk#69, ss_store_sk#71] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, hd_demo_sk#73] +(115) CometProject +Input [4]: [ss_sold_time_sk#64, ss_hdemo_sk#65, ss_store_sk#66, hd_demo_sk#68] +Arguments: [ss_sold_time_sk#64, ss_store_sk#66], [ss_sold_time_sk#64, ss_store_sk#66] -(124) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#74, t_hour#75, t_minute#76] +(116) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#69, t_hour#70, t_minute#71] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(125) CometFilter -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] -Condition : ((((isnotnull(t_hour#75) AND isnotnull(t_minute#76)) AND (t_hour#75 = 11)) AND (t_minute#76 < 30)) AND isnotnull(t_time_sk#74)) - -(126) CometProject -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] -Arguments: [t_time_sk#74], [t_time_sk#74] +(117) CometFilter +Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Condition : ((((isnotnull(t_hour#70) AND isnotnull(t_minute#71)) AND (t_hour#70 = 11)) AND (t_minute#71 < 30)) AND isnotnull(t_time_sk#69)) -(127) ColumnarToRow [codegen id : 26] -Input [1]: [t_time_sk#74] +(118) CometProject +Input [3]: [t_time_sk#69, t_hour#70, t_minute#71] +Arguments: [t_time_sk#69], [t_time_sk#69] -(128) BroadcastExchange -Input [1]: [t_time_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] +(119) CometBroadcastExchange +Input [1]: [t_time_sk#69] +Arguments: [t_time_sk#69] -(129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#69] -Right keys [1]: [t_time_sk#74] -Join type: Inner -Join condition: None +(120) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#64, ss_store_sk#66] +Right output [1]: [t_time_sk#69] +Arguments: [ss_sold_time_sk#64], [t_time_sk#69], Inner, BuildRight -(130) Project [codegen id : 28] -Output [1]: [ss_store_sk#71] -Input [3]: [ss_sold_time_sk#69, ss_store_sk#71, t_time_sk#74] +(121) CometProject +Input [3]: [ss_sold_time_sk#64, ss_store_sk#66, t_time_sk#69] +Arguments: [ss_store_sk#66], [ss_store_sk#66] -(131) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#77] +(122) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#72] -(132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#71] -Right keys [1]: [s_store_sk#77] -Join type: Inner -Join condition: None +(123) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#66] +Right output [1]: [s_store_sk#72] +Arguments: [ss_store_sk#66], [s_store_sk#72], Inner, BuildRight -(133) Project [codegen id : 28] -Output: [] -Input [2]: [ss_store_sk#71, s_store_sk#77] +(124) CometProject +Input [2]: [ss_store_sk#66, s_store_sk#72] -(134) HashAggregate [codegen id : 28] +(125) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#78] -Results [1]: [count#79] -(135) Exchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] +(126) ColumnarToRow [codegen id : 10] +Input [1]: [count#73] -(136) HashAggregate [codegen id : 29] -Input [1]: [count#79] +(127) Exchange +Input [1]: [count#73] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] + +(128) HashAggregate [codegen id : 11] +Input [1]: [count#73] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#80] -Results [1]: [count(1)#80 AS h11_to_11_30#81] +Aggregate Attributes [1]: [count(1)#74] +Results [1]: [count(1)#74 AS h11_to_11_30#75] -(137) BroadcastExchange -Input [1]: [h11_to_11_30#81] -Arguments: IdentityBroadcastMode, [plan_id=19] +(129) BroadcastExchange +Input [1]: [h11_to_11_30#75] +Arguments: IdentityBroadcastMode, [plan_id=11] -(138) BroadcastNestedLoopJoin [codegen id : 40] +(130) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(139) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] +(131) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(140) CometFilter -Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] -Condition : ((isnotnull(ss_hdemo_sk#83) AND isnotnull(ss_sold_time_sk#82)) AND isnotnull(ss_store_sk#84)) - -(141) CometProject -Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] -Arguments: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84], [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] +(132) CometFilter +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] +Condition : ((isnotnull(ss_hdemo_sk#77) AND isnotnull(ss_sold_time_sk#76)) AND isnotnull(ss_store_sk#78)) -(142) ColumnarToRow [codegen id : 33] -Input [3]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] +(133) CometProject +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, ss_sold_date_sk#79] +Arguments: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78], [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] -(143) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#86] +(134) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#80] -(144) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#83] -Right keys [1]: [hd_demo_sk#86] -Join type: Inner -Join condition: None +(135) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78] +Right output [1]: [hd_demo_sk#80] +Arguments: [ss_hdemo_sk#77], [hd_demo_sk#80], Inner, BuildRight -(145) Project [codegen id : 33] -Output [2]: [ss_sold_time_sk#82, ss_store_sk#84] -Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, hd_demo_sk#86] +(136) CometProject +Input [4]: [ss_sold_time_sk#76, ss_hdemo_sk#77, ss_store_sk#78, hd_demo_sk#80] +Arguments: [ss_sold_time_sk#76, ss_store_sk#78], [ss_sold_time_sk#76, ss_store_sk#78] -(146) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#87, t_hour#88, t_minute#89] +(137) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#81, t_hour#82, t_minute#83] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(147) CometFilter -Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] -Condition : ((((isnotnull(t_hour#88) AND isnotnull(t_minute#89)) AND (t_hour#88 = 11)) AND (t_minute#89 >= 30)) AND isnotnull(t_time_sk#87)) +(138) CometFilter +Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Condition : ((((isnotnull(t_hour#82) AND isnotnull(t_minute#83)) AND (t_hour#82 = 11)) AND (t_minute#83 >= 30)) AND isnotnull(t_time_sk#81)) -(148) CometProject -Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] -Arguments: [t_time_sk#87], [t_time_sk#87] +(139) CometProject +Input [3]: [t_time_sk#81, t_hour#82, t_minute#83] +Arguments: [t_time_sk#81], [t_time_sk#81] -(149) ColumnarToRow [codegen id : 31] -Input [1]: [t_time_sk#87] +(140) CometBroadcastExchange +Input [1]: [t_time_sk#81] +Arguments: [t_time_sk#81] -(150) BroadcastExchange -Input [1]: [t_time_sk#87] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] +(141) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#76, ss_store_sk#78] +Right output [1]: [t_time_sk#81] +Arguments: [ss_sold_time_sk#76], [t_time_sk#81], Inner, BuildRight -(151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#82] -Right keys [1]: [t_time_sk#87] -Join type: Inner -Join condition: None +(142) CometProject +Input [3]: [ss_sold_time_sk#76, ss_store_sk#78, t_time_sk#81] +Arguments: [ss_store_sk#78], [ss_store_sk#78] -(152) Project [codegen id : 33] -Output [1]: [ss_store_sk#84] -Input [3]: [ss_sold_time_sk#82, ss_store_sk#84, t_time_sk#87] +(143) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#84] -(153) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#90] +(144) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#78] +Right output [1]: [s_store_sk#84] +Arguments: [ss_store_sk#78], [s_store_sk#84], Inner, BuildRight -(154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#84] -Right keys [1]: [s_store_sk#90] -Join type: Inner -Join condition: None - -(155) Project [codegen id : 33] -Output: [] -Input [2]: [ss_store_sk#84, s_store_sk#90] +(145) CometProject +Input [2]: [ss_store_sk#78, s_store_sk#84] -(156) HashAggregate [codegen id : 33] +(146) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#91] -Results [1]: [count#92] -(157) Exchange -Input [1]: [count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=21] +(147) ColumnarToRow [codegen id : 12] +Input [1]: [count#85] -(158) HashAggregate [codegen id : 34] -Input [1]: [count#92] +(148) Exchange +Input [1]: [count#85] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] + +(149) HashAggregate [codegen id : 13] +Input [1]: [count#85] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#93] -Results [1]: [count(1)#93 AS h11_30_to_12#94] +Aggregate Attributes [1]: [count(1)#86] +Results [1]: [count(1)#86 AS h11_30_to_12#87] -(159) BroadcastExchange -Input [1]: [h11_30_to_12#94] -Arguments: IdentityBroadcastMode, [plan_id=22] +(150) BroadcastExchange +Input [1]: [h11_30_to_12#87] +Arguments: IdentityBroadcastMode, [plan_id=13] -(160) BroadcastNestedLoopJoin [codegen id : 40] +(151) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None -(161) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] +(152) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(162) CometFilter -Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] -Condition : ((isnotnull(ss_hdemo_sk#96) AND isnotnull(ss_sold_time_sk#95)) AND isnotnull(ss_store_sk#97)) - -(163) CometProject -Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] -Arguments: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97], [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] +(153) CometFilter +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] +Condition : ((isnotnull(ss_hdemo_sk#89) AND isnotnull(ss_sold_time_sk#88)) AND isnotnull(ss_store_sk#90)) -(164) ColumnarToRow [codegen id : 38] -Input [3]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] +(154) CometProject +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, ss_sold_date_sk#91] +Arguments: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90], [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90] -(165) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#99] +(155) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#92] -(166) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#96] -Right keys [1]: [hd_demo_sk#99] -Join type: Inner -Join condition: None +(156) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90] +Right output [1]: [hd_demo_sk#92] +Arguments: [ss_hdemo_sk#89], [hd_demo_sk#92], Inner, BuildRight -(167) Project [codegen id : 38] -Output [2]: [ss_sold_time_sk#95, ss_store_sk#97] -Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, hd_demo_sk#99] +(157) CometProject +Input [4]: [ss_sold_time_sk#88, ss_hdemo_sk#89, ss_store_sk#90, hd_demo_sk#92] +Arguments: [ss_sold_time_sk#88, ss_store_sk#90], [ss_sold_time_sk#88, ss_store_sk#90] -(168) Scan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#100, t_hour#101, t_minute#102] +(158) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#93, t_hour#94, t_minute#95] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(169) CometFilter -Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] -Condition : ((((isnotnull(t_hour#101) AND isnotnull(t_minute#102)) AND (t_hour#101 = 12)) AND (t_minute#102 < 30)) AND isnotnull(t_time_sk#100)) - -(170) CometProject -Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] -Arguments: [t_time_sk#100], [t_time_sk#100] +(159) CometFilter +Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Condition : ((((isnotnull(t_hour#94) AND isnotnull(t_minute#95)) AND (t_hour#94 = 12)) AND (t_minute#95 < 30)) AND isnotnull(t_time_sk#93)) -(171) ColumnarToRow [codegen id : 36] -Input [1]: [t_time_sk#100] +(160) CometProject +Input [3]: [t_time_sk#93, t_hour#94, t_minute#95] +Arguments: [t_time_sk#93], [t_time_sk#93] -(172) BroadcastExchange -Input [1]: [t_time_sk#100] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23] +(161) CometBroadcastExchange +Input [1]: [t_time_sk#93] +Arguments: [t_time_sk#93] -(173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#95] -Right keys [1]: [t_time_sk#100] -Join type: Inner -Join condition: None +(162) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#88, ss_store_sk#90] +Right output [1]: [t_time_sk#93] +Arguments: [ss_sold_time_sk#88], [t_time_sk#93], Inner, BuildRight -(174) Project [codegen id : 38] -Output [1]: [ss_store_sk#97] -Input [3]: [ss_sold_time_sk#95, ss_store_sk#97, t_time_sk#100] +(163) CometProject +Input [3]: [ss_sold_time_sk#88, ss_store_sk#90, t_time_sk#93] +Arguments: [ss_store_sk#90], [ss_store_sk#90] -(175) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#103] +(164) ReusedExchange [Reuses operator id: 19] +Output [1]: [s_store_sk#96] -(176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#97] -Right keys [1]: [s_store_sk#103] -Join type: Inner -Join condition: None +(165) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#90] +Right output [1]: [s_store_sk#96] +Arguments: [ss_store_sk#90], [s_store_sk#96], Inner, BuildRight -(177) Project [codegen id : 38] -Output: [] -Input [2]: [ss_store_sk#97, s_store_sk#103] +(166) CometProject +Input [2]: [ss_store_sk#90, s_store_sk#96] -(178) HashAggregate [codegen id : 38] +(167) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#104] -Results [1]: [count#105] -(179) Exchange -Input [1]: [count#105] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=24] +(168) ColumnarToRow [codegen id : 14] +Input [1]: [count#97] + +(169) Exchange +Input [1]: [count#97] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] -(180) HashAggregate [codegen id : 39] -Input [1]: [count#105] +(170) HashAggregate [codegen id : 15] +Input [1]: [count#97] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#106] -Results [1]: [count(1)#106 AS h12_to_12_30#107] +Aggregate Attributes [1]: [count(1)#98] +Results [1]: [count(1)#98 AS h12_to_12_30#99] -(181) BroadcastExchange -Input [1]: [h12_to_12_30#107] -Arguments: IdentityBroadcastMode, [plan_id=25] +(171) BroadcastExchange +Input [1]: [h12_to_12_30#99] +Arguments: IdentityBroadcastMode, [plan_id=15] -(182) BroadcastNestedLoopJoin [codegen id : 40] +(172) BroadcastNestedLoopJoin [codegen id : 16] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt index b497e0bab6..b846d25d83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (40) +WholeStageCodegen (16) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin @@ -9,257 +9,203 @@ WholeStageCodegen (40) HashAggregate [count] [count(1),h8_30_to_9,count] InputAdapter Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometBroadcastExchange #2 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #3 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange #4 + CometProject [s_store_sk] + CometFilter [s_store_name,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [count] [count(1),h9_to_9_30,count] InputAdapter Exchange #6 - WholeStageCodegen (8) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #7 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #8 - WholeStageCodegen (14) + WholeStageCodegen (5) HashAggregate [count] [count(1),h9_30_to_10,count] InputAdapter Exchange #9 - WholeStageCodegen (13) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #10 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #11 - WholeStageCodegen (19) + WholeStageCodegen (7) HashAggregate [count] [count(1),h10_to_10_30,count] InputAdapter Exchange #12 - WholeStageCodegen (18) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #13 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #14 - WholeStageCodegen (24) + WholeStageCodegen (9) HashAggregate [count] [count(1),h10_30_to_11,count] InputAdapter Exchange #15 - WholeStageCodegen (23) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (21) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #16 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #17 - WholeStageCodegen (29) + WholeStageCodegen (11) HashAggregate [count] [count(1),h11_to_11_30,count] InputAdapter Exchange #18 - WholeStageCodegen (28) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (26) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #19 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #20 - WholeStageCodegen (34) + WholeStageCodegen (13) HashAggregate [count] [count(1),h11_30_to_12,count] InputAdapter Exchange #21 - WholeStageCodegen (33) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #22 - WholeStageCodegen (31) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #22 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 InputAdapter BroadcastExchange #23 - WholeStageCodegen (39) + WholeStageCodegen (15) HashAggregate [count] [count(1),h12_to_12_30,count] InputAdapter Exchange #24 - WholeStageCodegen (38) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (14) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #25 - WholeStageCodegen (36) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - ReusedExchange [s_store_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #25 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index 3657266e23..49dab66f02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -1,31 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (27) -+- * Project (26) - +- * Filter (25) - +- Window (24) - +- * Sort (23) - +- Exchange (22) - +- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * Project (18) - +- * BroadcastHashJoin Inner BuildRight (17) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (7) - : : +- * ColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.store_sales (4) - : +- ReusedExchange (10) - +- BroadcastExchange (16) - +- * ColumnarToRow (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store (13) +TakeOrderedAndProject (28) ++- * Project (27) + +- * Filter (26) + +- Window (25) + +- * Sort (24) + +- Exchange (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * ColumnarToRow (20) + +- CometHashAggregate (19) + +- CometProject (18) + +- CometBroadcastHashJoin (17) + :- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (7) + : : +- CometBroadcastHashJoin (6) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- CometBroadcastExchange (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.store_sales (3) + : +- CometBroadcastExchange (11) + : +- CometProject (10) + : +- CometFilter (9) + : +- CometScan parquet spark_catalog.default.date_dim (8) + +- CometBroadcastExchange (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) (1) Scan parquet spark_catalog.default.item @@ -39,10 +40,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] Condition : (isnotnull(ss_item_sk#5) AND isnotnull(ss_store_sk#6)) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] +Right output [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +Arguments: [i_item_sk#1], [ss_item_sk#5], Inner, BuildRight -(9) Project [codegen id : 4] -Output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +(7) CometProject Input [8]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +Arguments: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8], [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] -(10) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_moy#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct + +(9) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((isnotnull(d_year#11) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) + +(10) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Arguments: [d_date_sk#10, d_moy#12], [d_date_sk#10, d_moy#12] -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(11) CometBroadcastExchange +Input [2]: [d_date_sk#10, d_moy#12] +Arguments: [d_date_sk#10, d_moy#12] -(12) Project [codegen id : 4] -Output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#11] -Input [8]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8, d_date_sk#10, d_moy#11] +(12) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +Right output [2]: [d_date_sk#10, d_moy#12] +Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight -(13) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +(13) CometProject +Input [8]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8, d_date_sk#10, d_moy#12] +Arguments: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#12], [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#12] + +(14) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] -Condition : isnotnull(s_store_sk#12) - -(15) ColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +(15) CometFilter +Input [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] +Condition : isnotnull(s_store_sk#13) -(16) BroadcastExchange -Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastExchange +Input [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] +Arguments: [s_store_sk#13, s_store_name#14, s_company_name#15] -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#12] +Right output [3]: [s_store_sk#13, s_store_name#14, s_company_name#15] +Arguments: [ss_store_sk#6], [s_store_sk#13], Inner, BuildRight -(18) Project [codegen id : 4] -Output [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#11, s_store_name#13, s_company_name#14] -Input [9]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] +(18) CometProject +Input [9]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#12, s_store_sk#13, s_store_name#14, s_company_name#15] +Arguments: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15], [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15] -(19) HashAggregate [codegen id : 4] -Input [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#11, s_store_name#13, s_company_name#14] -Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11] +(19) CometHashAggregate +Input [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15] +Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#7))] -Aggregate Attributes [1]: [sum#15] -Results [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] -(20) Exchange -Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(20) ColumnarToRow [codegen id : 1] +Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum#16] + +(21) Exchange +Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum#16] +Arguments: hashpartitioning(i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] -Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11] +(22) HashAggregate [codegen id : 2] +Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum#16] +Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12] Functions [1]: [sum(UnscaledValue(ss_sales_price#7))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#7))#17] -Results [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS _w0#19] +Results [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS _w0#19] -(22) Exchange -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(23) Exchange +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#18, _w0#19] +Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#14, s_company_name#15, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(23) Sort [codegen id : 6] -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -Arguments: [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST], false, 0 +(24) Sort [codegen id : 3] +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#18, _w0#19] +Arguments: [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST, s_company_name#15 ASC NULLS FIRST], false, 0 -(24) Window -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#4, i_brand#2, s_store_name#13, s_company_name#14] +(25) Window +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#18, _w0#19] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_name#14, s_company_name#15, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#4, i_brand#2, s_store_name#14, s_company_name#15] -(25) Filter [codegen id : 7] -Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] +(26) Filter [codegen id : 4] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#18, _w0#19, avg_monthly_sales#20] Condition : CASE WHEN NOT (avg_monthly_sales#20 = 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END -(26) Project [codegen id : 7] -Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] -Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] +(27) Project [codegen id : 4] +Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#18, avg_monthly_sales#20] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#18, _w0#19, avg_monthly_sales#20] -(27) TakeOrderedAndProject -Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] +(28) TakeOrderedAndProject +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#18, avg_monthly_sales#20] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#18, avg_monthly_sales#20] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (32) -+- * ColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan parquet spark_catalog.default.date_dim (28) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) -(28) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#21, d_moy#11] +(29) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(29) CometFilter -Input [3]: [d_date_sk#10, d_year#21, d_moy#11] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 1999)) AND isnotnull(d_date_sk#10)) +(30) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((isnotnull(d_year#11) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) -(30) CometProject -Input [3]: [d_date_sk#10, d_year#21, d_moy#11] -Arguments: [d_date_sk#10, d_moy#11], [d_date_sk#10, d_moy#11] +(31) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Arguments: [d_date_sk#10, d_moy#12], [d_date_sk#10, d_moy#12] -(31) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_moy#11] +(32) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#10, d_moy#12] -(32) BroadcastExchange -Input [2]: [d_date_sk#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) BroadcastExchange +Input [2]: [d_date_sk#10, d_moy#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index bb9e4e17e2..507ac8a91a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -1,50 +1,44 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,i_brand,s_store_name,s_company_name] InputAdapter Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometFilter [i_category,i_class,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt index fcfbca847a..dbea5e75de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt @@ -1,55 +1,51 @@ == Physical Plan == -* Project (51) -+- * BroadcastNestedLoopJoin Inner BuildRight (50) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (11) - : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : :- * ColumnarToRow (4) - : : : : +- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- BroadcastExchange (9) - : : : +- * ColumnarToRow (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan parquet spark_catalog.default.household_demographics (5) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan parquet spark_catalog.default.time_dim (12) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.web_page (19) - +- BroadcastExchange (49) - +- * HashAggregate (48) - +- Exchange (47) - +- * HashAggregate (46) - +- * Project (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * ColumnarToRow (32) - : : : +- CometProject (31) - : : : +- CometFilter (30) - : : : +- CometScan parquet spark_catalog.default.web_sales (29) - : : +- ReusedExchange (33) - : +- BroadcastExchange (40) - : +- * ColumnarToRow (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.time_dim (36) - +- ReusedExchange (43) +* Project (47) ++- * BroadcastNestedLoopJoin Inner BuildRight (46) + :- * HashAggregate (25) + : +- Exchange (24) + : +- * ColumnarToRow (23) + : +- CometHashAggregate (22) + : +- CometProject (21) + : +- CometBroadcastHashJoin (20) + : :- CometProject (15) + : : +- CometBroadcastHashJoin (14) + : : :- CometProject (9) + : : : +- CometBroadcastHashJoin (8) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.household_demographics (4) + : : +- CometBroadcastExchange (13) + : : +- CometProject (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.time_dim (10) + : +- CometBroadcastExchange (19) + : +- CometProject (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.web_page (16) + +- BroadcastExchange (45) + +- * HashAggregate (44) + +- Exchange (43) + +- * ColumnarToRow (42) + +- CometHashAggregate (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (31) + : : +- CometBroadcastHashJoin (30) + : : :- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.web_sales (26) + : : +- ReusedExchange (29) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometFilter (33) + : +- CometScan parquet spark_catalog.default.time_dim (32) + +- ReusedExchange (38) (1) Scan parquet spark_catalog.default.web_sales @@ -67,226 +63,202 @@ Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AN Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -(4) ColumnarToRow [codegen id : 4] -Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(5) Scan parquet spark_catalog.default.household_demographics +(4) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#5, hd_dep_count#6] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [2]: [hd_demo_sk#5, hd_dep_count#6] Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) -(7) CometProject +(6) CometProject Input [2]: [hd_demo_sk#5, hd_dep_count#6] Arguments: [hd_demo_sk#5], [hd_demo_sk#5] -(8) ColumnarToRow [codegen id : 1] +(7) CometBroadcastExchange Input [1]: [hd_demo_sk#5] +Arguments: [hd_demo_sk#5] -(9) BroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(8) CometBroadcastHashJoin +Left output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] +Right output [1]: [hd_demo_sk#5] +Arguments: [ws_ship_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight -(10) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_ship_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 4] -Output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] +(9) CometProject Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] +Arguments: [ws_sold_time_sk#1, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_web_page_sk#3] -(12) Scan parquet spark_catalog.default.time_dim +(10) Scan parquet spark_catalog.default.time_dim Output [2]: [t_time_sk#7, t_hour#8] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] ReadSchema: struct -(13) CometFilter +(11) CometFilter Input [2]: [t_time_sk#7, t_hour#8] Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) -(14) CometProject +(12) CometProject Input [2]: [t_time_sk#7, t_hour#8] Arguments: [t_time_sk#7], [t_time_sk#7] -(15) ColumnarToRow [codegen id : 2] -Input [1]: [t_time_sk#7] - -(16) BroadcastExchange +(13) CometBroadcastExchange Input [1]: [t_time_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [t_time_sk#7] -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#7] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] +Right output [1]: [t_time_sk#7] +Arguments: [ws_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight -(18) Project [codegen id : 4] -Output [1]: [ws_web_page_sk#3] +(15) CometProject Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] +Arguments: [ws_web_page_sk#3], [ws_web_page_sk#3] -(19) Scan parquet spark_catalog.default.web_page +(16) Scan parquet spark_catalog.default.web_page Output [2]: [wp_web_page_sk#9, wp_char_count#10] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] ReadSchema: struct -(20) CometFilter +(17) CometFilter Input [2]: [wp_web_page_sk#9, wp_char_count#10] Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) -(21) CometProject +(18) CometProject Input [2]: [wp_web_page_sk#9, wp_char_count#10] Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] -(22) ColumnarToRow [codegen id : 3] +(19) CometBroadcastExchange Input [1]: [wp_web_page_sk#9] +Arguments: [wp_web_page_sk#9] -(23) BroadcastExchange -Input [1]: [wp_web_page_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_web_page_sk#3] -Right keys [1]: [wp_web_page_sk#9] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [1]: [ws_web_page_sk#3] +Right output [1]: [wp_web_page_sk#9] +Arguments: [ws_web_page_sk#3], [wp_web_page_sk#9], Inner, BuildRight -(25) Project [codegen id : 4] -Output: [] +(21) CometProject Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] -(26) HashAggregate [codegen id : 4] +(22) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#11] -Results [1]: [count#12] -(27) Exchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(23) ColumnarToRow [codegen id : 1] +Input [1]: [count#11] + +(24) Exchange +Input [1]: [count#11] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 10] -Input [1]: [count#12] +(25) HashAggregate [codegen id : 4] +Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#13] -Results [1]: [count(1)#13 AS amc#14] +Aggregate Attributes [1]: [count(1)#12] +Results [1]: [count(1)#12 AS amc#13] -(29) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] +(26) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] -Condition : ((isnotnull(ws_ship_hdemo_sk#16) AND isnotnull(ws_sold_time_sk#15)) AND isnotnull(ws_web_page_sk#17)) +(27) CometFilter +Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] +Condition : ((isnotnull(ws_ship_hdemo_sk#15) AND isnotnull(ws_sold_time_sk#14)) AND isnotnull(ws_web_page_sk#16)) -(31) CometProject -Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] -Arguments: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17], [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] +(28) CometProject +Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, ws_sold_date_sk#17] +Arguments: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16], [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16] -(32) ColumnarToRow [codegen id : 8] -Input [3]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] +(29) ReusedExchange [Reuses operator id: 7] +Output [1]: [hd_demo_sk#18] -(33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#19] +(30) CometBroadcastHashJoin +Left output [3]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16] +Right output [1]: [hd_demo_sk#18] +Arguments: [ws_ship_hdemo_sk#15], [hd_demo_sk#18], Inner, BuildRight -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#16] -Right keys [1]: [hd_demo_sk#19] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#15, ws_web_page_sk#17] -Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, hd_demo_sk#19] +(31) CometProject +Input [4]: [ws_sold_time_sk#14, ws_ship_hdemo_sk#15, ws_web_page_sk#16, hd_demo_sk#18] +Arguments: [ws_sold_time_sk#14, ws_web_page_sk#16], [ws_sold_time_sk#14, ws_web_page_sk#16] -(36) Scan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_hour#21] +(32) Scan parquet spark_catalog.default.time_dim +Output [2]: [t_time_sk#19, t_hour#20] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct -(37) CometFilter -Input [2]: [t_time_sk#20, t_hour#21] -Condition : (((isnotnull(t_hour#21) AND (t_hour#21 >= 19)) AND (t_hour#21 <= 20)) AND isnotnull(t_time_sk#20)) +(33) CometFilter +Input [2]: [t_time_sk#19, t_hour#20] +Condition : (((isnotnull(t_hour#20) AND (t_hour#20 >= 19)) AND (t_hour#20 <= 20)) AND isnotnull(t_time_sk#19)) -(38) CometProject -Input [2]: [t_time_sk#20, t_hour#21] -Arguments: [t_time_sk#20], [t_time_sk#20] +(34) CometProject +Input [2]: [t_time_sk#19, t_hour#20] +Arguments: [t_time_sk#19], [t_time_sk#19] -(39) ColumnarToRow [codegen id : 6] -Input [1]: [t_time_sk#20] +(35) CometBroadcastExchange +Input [1]: [t_time_sk#19] +Arguments: [t_time_sk#19] -(40) BroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) CometBroadcastHashJoin +Left output [2]: [ws_sold_time_sk#14, ws_web_page_sk#16] +Right output [1]: [t_time_sk#19] +Arguments: [ws_sold_time_sk#14], [t_time_sk#19], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#15] -Right keys [1]: [t_time_sk#20] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 8] -Output [1]: [ws_web_page_sk#17] -Input [3]: [ws_sold_time_sk#15, ws_web_page_sk#17, t_time_sk#20] +(37) CometProject +Input [3]: [ws_sold_time_sk#14, ws_web_page_sk#16, t_time_sk#19] +Arguments: [ws_web_page_sk#16], [ws_web_page_sk#16] -(43) ReusedExchange [Reuses operator id: 23] -Output [1]: [wp_web_page_sk#22] +(38) ReusedExchange [Reuses operator id: 19] +Output [1]: [wp_web_page_sk#21] -(44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#17] -Right keys [1]: [wp_web_page_sk#22] -Join type: Inner -Join condition: None +(39) CometBroadcastHashJoin +Left output [1]: [ws_web_page_sk#16] +Right output [1]: [wp_web_page_sk#21] +Arguments: [ws_web_page_sk#16], [wp_web_page_sk#21], Inner, BuildRight -(45) Project [codegen id : 8] -Output: [] -Input [2]: [ws_web_page_sk#17, wp_web_page_sk#22] +(40) CometProject +Input [2]: [ws_web_page_sk#16, wp_web_page_sk#21] -(46) HashAggregate [codegen id : 8] +(41) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#23] -Results [1]: [count#24] -(47) Exchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +(42) ColumnarToRow [codegen id : 2] +Input [1]: [count#22] + +(43) Exchange +Input [1]: [count#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(48) HashAggregate [codegen id : 9] -Input [1]: [count#24] +(44) HashAggregate [codegen id : 3] +Input [1]: [count#22] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#25] -Results [1]: [count(1)#25 AS pmc#26] +Aggregate Attributes [1]: [count(1)#23] +Results [1]: [count(1)#23 AS pmc#24] -(49) BroadcastExchange -Input [1]: [pmc#26] -Arguments: IdentityBroadcastMode, [plan_id=7] +(45) BroadcastExchange +Input [1]: [pmc#24] +Arguments: IdentityBroadcastMode, [plan_id=3] -(50) BroadcastNestedLoopJoin [codegen id : 10] +(46) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(51) Project [codegen id : 10] -Output [1]: [(cast(amc#14 as decimal(15,4)) / cast(pmc#26 as decimal(15,4))) AS am_pm_ratio#27] -Input [2]: [amc#14, pmc#26] +(47) Project [codegen id : 4] +Output [1]: [(cast(amc#13 as decimal(15,4)) / cast(pmc#24 as decimal(15,4))) AS am_pm_ratio#25] +Input [2]: [amc#13, pmc#24] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt index c4e04b06bc..50c8494fb5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt @@ -1,74 +1,56 @@ -WholeStageCodegen (10) +WholeStageCodegen (4) Project [amc,pmc] BroadcastNestedLoopJoin HashAggregate [count] [count(1),amc,count] InputAdapter Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [wp_web_page_sk] - CometFilter [wp_char_count,wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] + CometBroadcastExchange #2 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange #3 + CometProject [t_time_sk] + CometFilter [t_hour,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + CometBroadcastExchange #4 + CometProject [wp_web_page_sk] + CometFilter [wp_char_count,wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [count] [count(1),pmc,count] InputAdapter Exchange #6 - WholeStageCodegen (8) - HashAggregate [count,count] - Project - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [ws_sold_time_sk,ws_web_page_sk] - BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - InputAdapter - ReusedExchange [hd_demo_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - InputAdapter - ReusedExchange [wp_web_page_sk] #4 + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange #7 + CometProject [t_time_sk] + CometFilter [t_hour,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt index 61f35489a2..cd2e8cfb00 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt @@ -1,47 +1,45 @@ == Physical Plan == -* Sort (43) -+- Exchange (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (18) - : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : :- * Project (12) - : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * ColumnarToRow (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.call_center (1) - : : : : : +- BroadcastExchange (7) - : : : : : +- * ColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (4) - : : : : +- ReusedExchange (10) - : : : +- BroadcastExchange (16) - : : : +- * ColumnarToRow (15) - : : : +- CometFilter (14) - : : : +- CometScan parquet spark_catalog.default.customer (13) - : : +- BroadcastExchange (23) - : : +- * ColumnarToRow (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan parquet spark_catalog.default.customer_address (19) - : +- BroadcastExchange (29) - : +- * ColumnarToRow (28) - : +- CometFilter (27) - : +- CometScan parquet spark_catalog.default.customer_demographics (26) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.household_demographics (32) +* Sort (41) ++- Exchange (40) + +- * HashAggregate (39) + +- Exchange (38) + +- * ColumnarToRow (37) + +- CometHashAggregate (36) + +- CometProject (35) + +- CometBroadcastHashJoin (34) + :- CometProject (29) + : +- CometBroadcastHashJoin (28) + : :- CometProject (24) + : : +- CometBroadcastHashJoin (23) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (13) + : : : : +- CometBroadcastHashJoin (12) + : : : : :- CometProject (7) + : : : : : +- CometBroadcastHashJoin (6) + : : : : : :- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.call_center (1) + : : : : : +- CometBroadcastExchange (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (3) + : : : : +- CometBroadcastExchange (11) + : : : : +- CometProject (10) + : : : : +- CometFilter (9) + : : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : : +- CometBroadcastExchange (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.customer (14) + : : +- CometBroadcastExchange (22) + : : +- CometProject (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.customer_address (19) + : +- CometBroadcastExchange (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.customer_demographics (25) + +- CometBroadcastExchange (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.household_demographics (30) (1) Scan parquet spark_catalog.default.call_center @@ -55,10 +53,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] Condition : (isnotnull(cr_call_center_sk#6) AND isnotnull(cr_returning_customer_sk#5)) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +Arguments: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] -(8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cc_call_center_sk#1] -Right keys [1]: [cr_call_center_sk#6] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] +Right output [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] +Arguments: [cc_call_center_sk#1], [cr_call_center_sk#6], Inner, BuildRight -(9) Project [codegen id : 7] -Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8] +(7) CometProject Input [8]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8] + +(8) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] +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 + +(9) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 1998)) AND (d_moy#12 = 11)) AND isnotnull(d_date_sk#10)) + +(10) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Arguments: [d_date_sk#10], [d_date_sk#10] -(10) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#10] +(11) CometBroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: [d_date_sk#10] -(11) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returned_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8] +Right output [1]: [d_date_sk#10] +Arguments: [cr_returned_date_sk#8], [d_date_sk#10], Inner, BuildRight -(12) Project [codegen id : 7] -Output [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7] +(13) CometProject Input [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8, d_date_sk#10] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7] -(13) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +(14) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] 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 -(14) CometFilter -Input [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] -Condition : (((isnotnull(c_customer_sk#11) AND isnotnull(c_current_addr_sk#14)) AND isnotnull(c_current_cdemo_sk#12)) AND isnotnull(c_current_hdemo_sk#13)) - -(15) ColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +(15) CometFilter +Input [4]: [c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] +Condition : (((isnotnull(c_customer_sk#13) AND isnotnull(c_current_addr_sk#16)) AND isnotnull(c_current_cdemo_sk#14)) AND isnotnull(c_current_hdemo_sk#15)) -(16) BroadcastExchange -Input [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastExchange +Input [4]: [c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] +Arguments: [c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] -(17) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returning_customer_sk#5] -Right keys [1]: [c_customer_sk#11] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7] +Right output [4]: [c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] +Arguments: [cr_returning_customer_sk#5], [c_customer_sk#13], Inner, BuildRight -(18) Project [codegen id : 7] -Output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] -Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +(18) CometProject +Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, c_customer_sk#13, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] (19) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#15, ca_gmt_offset#16] +Output [2]: [ca_address_sk#17, ca_gmt_offset#18] 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 (20) CometFilter -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] -Condition : ((isnotnull(ca_gmt_offset#16) AND (ca_gmt_offset#16 = -7.00)) AND isnotnull(ca_address_sk#15)) +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)) (21) CometProject -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] -Arguments: [ca_address_sk#15], [ca_address_sk#15] +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +Arguments: [ca_address_sk#17], [ca_address_sk#17] -(22) ColumnarToRow [codegen id : 4] -Input [1]: [ca_address_sk#15] +(22) CometBroadcastExchange +Input [1]: [ca_address_sk#17] +Arguments: [ca_address_sk#17] -(23) BroadcastExchange -Input [1]: [ca_address_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#14] -Right keys [1]: [ca_address_sk#15] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16] +Right output [1]: [ca_address_sk#17] +Arguments: [c_current_addr_sk#16], [ca_address_sk#17], Inner, BuildRight -(25) Project [codegen id : 7] -Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13] -Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14, ca_address_sk#15] +(24) CometProject +Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15, c_current_addr_sk#16, ca_address_sk#17] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15] -(26) Scan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +(25) Scan parquet spark_catalog.default.customer_demographics +Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] 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 -(27) CometFilter -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] -Condition : ((((cd_marital_status#18 = M) AND (cd_education_status#19 = Unknown )) OR ((cd_marital_status#18 = W) AND (cd_education_status#19 = Advanced Degree ))) AND isnotnull(cd_demo_sk#17)) - -(28) ColumnarToRow [codegen id : 5] -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +(26) CometFilter +Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Condition : ((((cd_marital_status#20 = M) AND (cd_education_status#21 = Unknown )) OR ((cd_marital_status#20 = W) AND (cd_education_status#21 = Advanced Degree ))) AND isnotnull(cd_demo_sk#19)) -(29) BroadcastExchange -Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(27) CometBroadcastExchange +Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Arguments: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -(30) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#12] -Right keys [1]: [cd_demo_sk#17] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15] +Right output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Arguments: [c_current_cdemo_sk#14], [cd_demo_sk#19], Inner, BuildRight -(31) Project [codegen id : 7] -Output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#13, cd_marital_status#18, cd_education_status#19] -Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13, cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +(29) CometProject +Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#14, c_current_hdemo_sk#15, cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#15, cd_marital_status#20, cd_education_status#21], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#15, cd_marital_status#20, cd_education_status#21] -(32) Scan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] +(30) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#22, hd_buy_potential#23] 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 -(33) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((isnotnull(hd_buy_potential#21) AND StartsWith(hd_buy_potential#21, Unknown)) AND isnotnull(hd_demo_sk#20)) +(31) CometFilter +Input [2]: [hd_demo_sk#22, hd_buy_potential#23] +Condition : ((isnotnull(hd_buy_potential#23) AND StartsWith(hd_buy_potential#23, Unknown)) AND isnotnull(hd_demo_sk#22)) -(34) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] +(32) CometProject +Input [2]: [hd_demo_sk#22, hd_buy_potential#23] +Arguments: [hd_demo_sk#22], [hd_demo_sk#22] -(35) ColumnarToRow [codegen id : 6] -Input [1]: [hd_demo_sk#20] +(33) CometBroadcastExchange +Input [1]: [hd_demo_sk#22] +Arguments: [hd_demo_sk#22] -(36) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(34) CometBroadcastHashJoin +Left output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#15, cd_marital_status#20, cd_education_status#21] +Right output [1]: [hd_demo_sk#22] +Arguments: [c_current_hdemo_sk#15], [hd_demo_sk#22], Inner, BuildRight -(37) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#13] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None +(35) CometProject +Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#15, cd_marital_status#20, cd_education_status#21, hd_demo_sk#22] +Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21] -(38) Project [codegen id : 7] -Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#18, cd_education_status#19] -Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#13, cd_marital_status#18, cd_education_status#19, hd_demo_sk#20] - -(39) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#18, cd_education_status#19] -Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19] +(36) CometHashAggregate +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21] +Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21] Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#7))] -Aggregate Attributes [1]: [sum#22] -Results [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] -(40) Exchange -Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] -Arguments: hashpartitioning(cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(37) ColumnarToRow [codegen id : 1] +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21, sum#24] + +(38) Exchange +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21, sum#24] +Arguments: hashpartitioning(cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(41) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] -Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19] +(39) HashAggregate [codegen id : 2] +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21, sum#24] +Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#20, cd_education_status#21] Functions [1]: [sum(UnscaledValue(cr_net_loss#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#7))#24] -Results [4]: [cc_call_center_id#2 AS Call_Center#25, cc_name#3 AS Call_Center_Name#26, cc_manager#4 AS Manager#27, MakeDecimal(sum(UnscaledValue(cr_net_loss#7))#24,17,2) AS Returns_Loss#28] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#7))#25] +Results [4]: [cc_call_center_id#2 AS Call_Center#26, cc_name#3 AS Call_Center_Name#27, cc_manager#4 AS Manager#28, MakeDecimal(sum(UnscaledValue(cr_net_loss#7))#25,17,2) AS Returns_Loss#29] -(42) Exchange -Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] -Arguments: rangepartitioning(Returns_Loss#28 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(40) Exchange +Input [4]: [Call_Center#26, Call_Center_Name#27, Manager#28, Returns_Loss#29] +Arguments: rangepartitioning(Returns_Loss#29 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(43) Sort [codegen id : 9] -Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] -Arguments: [Returns_Loss#28 DESC NULLS LAST], true, 0 +(41) Sort [codegen id : 3] +Input [4]: [Call_Center#26, Call_Center_Name#27, Manager#28, Returns_Loss#29] +Arguments: [Returns_Loss#29 DESC NULLS LAST], true, 0 ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (48) -+- * ColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan parquet spark_catalog.default.date_dim (44) +Subquery:1 Hosting operator id = 3 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) -(44) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#29, d_moy#30] +(42) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#11, d_moy#12] 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 -(45) CometFilter -Input [3]: [d_date_sk#10, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 1998)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#10)) +(43) CometFilter +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] +Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 1998)) AND (d_moy#12 = 11)) AND isnotnull(d_date_sk#10)) -(46) CometProject -Input [3]: [d_date_sk#10, d_year#29, d_moy#30] +(44) CometProject +Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(47) ColumnarToRow [codegen id : 1] +(45) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(48) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index e5d62e3c0b..b415eb5c43 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -1,73 +1,55 @@ -WholeStageCodegen (9) +WholeStageCodegen (3) Sort [Returns_Loss] InputAdapter Exchange [Returns_Loss] #1 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] InputAdapter Exchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - WholeStageCodegen (7) - 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 [c_current_hdemo_sk,hd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] + CometBroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] CometFilter [cc_call_center_sk] CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cr_call_center_sk,cr_returning_customer_sk] - CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_gmt_offset,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange #3 + CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometBroadcastExchange #7 + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange #8 + CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastExchange #9 + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index 5f1f961686..074fe2b999 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -1,33 +1,35 @@ == Physical Plan == -* HashAggregate (29) -+- Exchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- BroadcastExchange (8) - : : +- * ColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * HashAggregate (19) - : +- Exchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * ColumnarToRow (13) - : : +- CometFilter (12) - : : +- CometScan parquet spark_catalog.default.web_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (24) +* HashAggregate (31) ++- Exchange (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * ColumnarToRow (9) + : : +- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.item (3) + : +- BroadcastExchange (23) + : +- * Filter (22) + : +- * HashAggregate (21) + : +- Exchange (20) + : +- * ColumnarToRow (19) + : +- CometHashAggregate (18) + : +- CometProject (17) + : +- CometBroadcastHashJoin (16) + : :- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.web_sales (10) + : +- CometBroadcastExchange (15) + : +- CometProject (14) + : +- CometFilter (13) + : +- CometScan parquet spark_catalog.default.date_dim (12) + +- ReusedExchange (26) (1) Scan parquet spark_catalog.default.web_sales @@ -42,42 +44,38 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) -(3) ColumnarToRow [codegen id : 6] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#5, i_manufact_id#6] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [i_item_sk#5, i_manufact_id#6] Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) -(6) CometProject +(5) CometProject Input [2]: [i_item_sk#5, i_manufact_id#6] Arguments: [i_item_sk#5], [i_item_sk#5] -(7) ColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [1]: [i_item_sk#5] +Arguments: [i_item_sk#5] -(8) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Right output [1]: [i_item_sk#5] +Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] +(8) CometProject Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] +Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5], [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] + +(9) ColumnarToRow [codegen id : 4] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -(11) Scan parquet spark_catalog.default.web_sales +(10) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] @@ -85,125 +83,138 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(12) CometFilter +(11) CometFilter Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] Condition : isnotnull(ws_item_sk#7) -(13) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] +(12) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#11] +(13) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(15) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] -(16) Project [codegen id : 3] -Output [2]: [ws_item_sk#7, ws_ext_discount_amt#8] +(16) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] +Right output [1]: [d_date_sk#11] +Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight + +(17) CometProject Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#11] +Arguments: [ws_item_sk#7, ws_ext_discount_amt#8], [ws_item_sk#7, ws_ext_discount_amt#8] -(17) HashAggregate [codegen id : 3] +(18) CometHashAggregate Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] Keys [1]: [ws_item_sk#7] Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] -Aggregate Attributes [2]: [sum#12, count#13] -Results [3]: [ws_item_sk#7, sum#14, count#15] -(18) Exchange -Input [3]: [ws_item_sk#7, sum#14, count#15] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(19) ColumnarToRow [codegen id : 1] +Input [3]: [ws_item_sk#7, sum#13, count#14] + +(20) Exchange +Input [3]: [ws_item_sk#7, sum#13, count#14] +Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(19) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#7, sum#14, count#15] +(21) HashAggregate [codegen id : 2] +Input [3]: [ws_item_sk#7, sum#13, count#14] Keys [1]: [ws_item_sk#7] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#16] -Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#15] +Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -(20) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) +(22) Filter [codegen id : 2] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) -(21) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] +(23) BroadcastExchange +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=2] -(22) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#5] Right keys [1]: [ws_item_sk#7] Join type: Inner -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#16) -(23) Project [codegen id : 6] +(25) Project [codegen id : 4] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -(24) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#18] +(26) ReusedExchange [Reuses operator id: 36] +Output [1]: [d_date_sk#17] -(25) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#18] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#17] -(27) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 4] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#19] -Results [1]: [sum#20] +Aggregate Attributes [1]: [sum#18] +Results [1]: [sum#19] -(28) Exchange -Input [1]: [sum#20] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(30) Exchange +Input [1]: [sum#19] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(29) HashAggregate [codegen id : 7] -Input [1]: [sum#20] +(31) HashAggregate [codegen id : 5] +Input [1]: [sum#19] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#21] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#21,17,2) AS Excess Discount Amount #22] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#20] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#20,17,2) AS Excess Discount Amount #21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (34) -+- * ColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan parquet spark_catalog.default.date_dim (30) +BroadcastExchange (36) ++- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(30) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#23] +(32) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#17, d_date#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [d_date_sk#18, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) +(33) CometFilter +Input [2]: [d_date_sk#17, d_date#22] +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#22 <= 2000-04-26)) AND isnotnull(d_date_sk#17)) -(32) CometProject -Input [2]: [d_date_sk#18, d_date#23] -Arguments: [d_date_sk#18], [d_date_sk#18] +(34) CometProject +Input [2]: [d_date_sk#17, d_date#22] +Arguments: [d_date_sk#17], [d_date_sk#17] -(33) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] +(35) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#17] -(34) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) BroadcastExchange +Input [1]: [d_date_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 10 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index a5e724c1ff..e2f498028a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -1,17 +1,17 @@ -WholeStageCodegen (7) +WholeStageCodegen (5) HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] InputAdapter Exchange #1 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ws_ext_discount_amt] [sum,sum] Project [ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_ext_discount_amt,ws_sold_date_sk] BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] - Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk,ws_ext_discount_amt] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -22,31 +22,29 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_manufact_id,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange #3 + CometProject [i_item_sk] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Filter [(1.3 * avg(ws_ext_discount_amt))] HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] InputAdapter Exchange [ws_item_sk] #5 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] - Project [ws_item_sk,ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ws_item_sk,ws_ext_discount_amt] + CometProject [ws_item_sk,ws_ext_discount_amt] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt index 60c262e9c4..4bc24750f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt @@ -1,32 +1,29 @@ == Physical Plan == -* HashAggregate (28) -+- Exchange (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * ColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- BroadcastExchange (9) - : : +- * ColumnarToRow (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.household_demographics (5) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometProject (14) - : +- CometFilter (13) - : +- CometScan parquet spark_catalog.default.time_dim (12) - +- BroadcastExchange (23) - +- * ColumnarToRow (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan parquet spark_catalog.default.store (19) +* HashAggregate (25) ++- Exchange (24) + +- * ColumnarToRow (23) + +- CometHashAggregate (22) + +- CometProject (21) + +- CometBroadcastHashJoin (20) + :- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.household_demographics (4) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.time_dim (10) + +- CometBroadcastExchange (19) + +- CometProject (18) + +- CometFilter (17) + +- CometScan parquet spark_catalog.default.store (16) (1) Scan parquet spark_catalog.default.store_sales @@ -44,120 +41,105 @@ Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isn Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -(4) ColumnarToRow [codegen id : 4] -Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(5) Scan parquet spark_catalog.default.household_demographics +(4) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#5, hd_dep_count#6] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] ReadSchema: struct -(6) CometFilter +(5) CometFilter Input [2]: [hd_demo_sk#5, hd_dep_count#6] Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) -(7) CometProject +(6) CometProject Input [2]: [hd_demo_sk#5, hd_dep_count#6] Arguments: [hd_demo_sk#5], [hd_demo_sk#5] -(8) ColumnarToRow [codegen id : 1] -Input [1]: [hd_demo_sk#5] - -(9) BroadcastExchange +(7) CometBroadcastExchange Input [1]: [hd_demo_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [hd_demo_sk#5] -(10) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Right output [1]: [hd_demo_sk#5] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight -(11) Project [codegen id : 4] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +(9) CometProject Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] -(12) Scan parquet spark_catalog.default.time_dim +(10) Scan parquet spark_catalog.default.time_dim Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct -(13) CometFilter +(11) CometFilter Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) -(14) CometProject +(12) CometProject Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] Arguments: [t_time_sk#7], [t_time_sk#7] -(15) ColumnarToRow [codegen id : 2] -Input [1]: [t_time_sk#7] - -(16) BroadcastExchange +(13) CometBroadcastExchange Input [1]: [t_time_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [t_time_sk#7] -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#7] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Right output [1]: [t_time_sk#7] +Arguments: [ss_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight -(18) Project [codegen id : 4] -Output [1]: [ss_store_sk#3] +(15) CometProject Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] +Arguments: [ss_store_sk#3], [ss_store_sk#3] -(19) Scan parquet spark_catalog.default.store +(16) Scan parquet spark_catalog.default.store Output [2]: [s_store_sk#10, s_store_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] ReadSchema: struct -(20) CometFilter +(17) CometFilter Input [2]: [s_store_sk#10, s_store_name#11] Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) -(21) CometProject +(18) CometProject Input [2]: [s_store_sk#10, s_store_name#11] Arguments: [s_store_sk#10], [s_store_sk#10] -(22) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#10] - -(23) BroadcastExchange +(19) CometBroadcastExchange Input [1]: [s_store_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [s_store_sk#10] -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#10] -Join type: Inner -Join condition: None +(20) CometBroadcastHashJoin +Left output [1]: [ss_store_sk#3] +Right output [1]: [s_store_sk#10] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(25) Project [codegen id : 4] -Output: [] +(21) CometProject Input [2]: [ss_store_sk#3, s_store_sk#10] -(26) HashAggregate [codegen id : 4] +(22) CometHashAggregate Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#12] -Results [1]: [count#13] -(27) Exchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(23) ColumnarToRow [codegen id : 1] +Input [1]: [count#12] + +(24) Exchange +Input [1]: [count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 5] -Input [1]: [count#13] +(25) HashAggregate [codegen id : 2] +Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#14] -Results [1]: [count(1)#14 AS count(1)#15] +Aggregate Attributes [1]: [count(1)#13] +Results [1]: [count(1)#13 AS count(1)#14] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt index d1438f48eb..6149152260 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt @@ -1,41 +1,29 @@ -WholeStageCodegen (5) +WholeStageCodegen (2) HashAggregate [count] [count(1),count(1),count] InputAdapter Exchange #1 - WholeStageCodegen (4) - HashAggregate [count,count] - Project - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk] - BroadcastHashJoin [ss_sold_time_sk,t_time_sk] - Project [ss_sold_time_sk,ss_store_sk] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_hour,t_minute,t_time_sk] - CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_name,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + CometBroadcastExchange #2 + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange #3 + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange #4 + CometProject [s_store_sk] + CometFilter [s_store_name,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt index 66ccf4f223..7508405a85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt @@ -1,27 +1,30 @@ == Physical Plan == -* HashAggregate (23) -+- Exchange (22) - +- * HashAggregate (21) - +- * Project (20) - +- * SortMergeJoin FullOuter (19) - :- * Sort (9) - : +- * HashAggregate (8) - : +- Exchange (7) - : +- * HashAggregate (6) - : +- * Project (5) - : +- * BroadcastHashJoin Inner BuildRight (4) - : :- * ColumnarToRow (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (3) - +- * Sort (18) - +- * HashAggregate (17) - +- Exchange (16) - +- * HashAggregate (15) - +- * Project (14) - +- * BroadcastHashJoin Inner BuildRight (13) - :- * ColumnarToRow (11) - : +- CometScan parquet spark_catalog.default.catalog_sales (10) - +- ReusedExchange (12) +* HashAggregate (26) ++- Exchange (25) + +- * HashAggregate (24) + +- * Project (23) + +- * SortMergeJoin FullOuter (22) + :- * Sort (12) + : +- * HashAggregate (11) + : +- Exchange (10) + : +- * ColumnarToRow (9) + : +- CometHashAggregate (8) + : +- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (5) + : +- CometProject (4) + : +- CometFilter (3) + : +- CometScan parquet spark_catalog.default.date_dim (2) + +- * Sort (21) + +- * HashAggregate (20) + +- Exchange (19) + +- * ColumnarToRow (18) + +- CometHashAggregate (17) + +- CometProject (16) + +- CometBroadcastHashJoin (15) + :- CometScan parquet spark_catalog.default.catalog_sales (13) + +- ReusedExchange (14) (1) Scan parquet spark_catalog.default.store_sales @@ -31,149 +34,159 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] +(2) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(3) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#5] +(3) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(4) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(4) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(5) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] -(5) Project [codegen id : 2] -Output [2]: [ss_item_sk#1, ss_customer_sk#2] +(6) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(7) CometProject Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] +Arguments: [ss_item_sk#1, ss_customer_sk#2], [ss_item_sk#1, ss_customer_sk#2] -(6) HashAggregate [codegen id : 2] +(8) CometHashAggregate Input [2]: [ss_item_sk#1, ss_customer_sk#2] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] -Aggregate Attributes: [] -Results [2]: [ss_customer_sk#2, ss_item_sk#1] -(7) Exchange +(9) ColumnarToRow [codegen id : 1] +Input [2]: [ss_customer_sk#2, ss_item_sk#1] + +(10) Exchange Input [2]: [ss_customer_sk#2, ss_item_sk#1] Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(8) HashAggregate [codegen id : 3] +(11) HashAggregate [codegen id : 2] Input [2]: [ss_customer_sk#2, ss_item_sk#1] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] Aggregate Attributes: [] -Results [2]: [ss_customer_sk#2 AS customer_sk#6, ss_item_sk#1 AS item_sk#7] +Results [2]: [ss_customer_sk#2 AS customer_sk#7, ss_item_sk#1 AS item_sk#8] -(9) Sort [codegen id : 3] -Input [2]: [customer_sk#6, item_sk#7] -Arguments: [customer_sk#6 ASC NULLS FIRST, item_sk#7 ASC NULLS FIRST], false, 0 +(12) Sort [codegen id : 2] +Input [2]: [customer_sk#7, item_sk#8] +Arguments: [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST], false, 0 -(10) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] +(13) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 5] -Input [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] +(14) ReusedExchange [Reuses operator id: 5] +Output [1]: [d_date_sk#13] -(12) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#12] +(15) CometBroadcastHashJoin +Left output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(13) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 5] -Output [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Input [4]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10, d_date_sk#12] +(16) CometProject +Input [4]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11, d_date_sk#13] +Arguments: [cs_bill_customer_sk#9, cs_item_sk#10], [cs_bill_customer_sk#9, cs_item_sk#10] -(15) HashAggregate [codegen id : 5] -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] +(17) CometHashAggregate +Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] +Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Functions: [] -Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -(16) Exchange -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Arguments: hashpartitioning(cs_bill_customer_sk#8, cs_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(18) ColumnarToRow [codegen id : 3] +Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] + +(19) Exchange +Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] +Arguments: hashpartitioning(cs_bill_customer_sk#9, cs_item_sk#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) HashAggregate [codegen id : 6] -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] +(20) HashAggregate [codegen id : 4] +Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] +Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#8 AS customer_sk#13, cs_item_sk#9 AS item_sk#14] +Results [2]: [cs_bill_customer_sk#9 AS customer_sk#14, cs_item_sk#10 AS item_sk#15] -(18) Sort [codegen id : 6] -Input [2]: [customer_sk#13, item_sk#14] -Arguments: [customer_sk#13 ASC NULLS FIRST, item_sk#14 ASC NULLS FIRST], false, 0 +(21) Sort [codegen id : 4] +Input [2]: [customer_sk#14, item_sk#15] +Arguments: [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST], false, 0 -(19) SortMergeJoin [codegen id : 7] -Left keys [2]: [customer_sk#6, item_sk#7] -Right keys [2]: [customer_sk#13, item_sk#14] +(22) SortMergeJoin [codegen id : 5] +Left keys [2]: [customer_sk#7, item_sk#8] +Right keys [2]: [customer_sk#14, item_sk#15] Join type: FullOuter Join condition: None -(20) Project [codegen id : 7] -Output [2]: [customer_sk#6, customer_sk#13] -Input [4]: [customer_sk#6, item_sk#7, customer_sk#13, item_sk#14] +(23) Project [codegen id : 5] +Output [2]: [customer_sk#7, customer_sk#14] +Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] -(21) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#6, customer_sk#13] +(24) HashAggregate [codegen id : 5] +Input [2]: [customer_sk#7, customer_sk#14] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#15, sum#16, sum#17] -Results [3]: [sum#18, sum#19, sum#20] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#16, sum#17, sum#18] +Results [3]: [sum#19, sum#20, sum#21] -(22) Exchange -Input [3]: [sum#18, sum#19, sum#20] +(25) Exchange +Input [3]: [sum#19, sum#20, sum#21] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(23) HashAggregate [codegen id : 8] -Input [3]: [sum#18, sum#19, sum#20] +(26) HashAggregate [codegen id : 6] +Input [3]: [sum#19, sum#20, sum#21] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END)#21, sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#22, sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#23] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END)#21 AS store_only#24, sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#22 AS catalog_only#25, sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#23 AS store_and_catalog#26] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END)#22, sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#23, sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#24] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END)#22 AS store_only#25, sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#23 AS catalog_only#26, sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)#24 AS store_and_catalog#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * ColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan parquet spark_catalog.default.date_dim (24) +BroadcastExchange (31) ++- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(24) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#27] +(27) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(25) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#27] -Condition : (((isnotnull(d_month_seq#27) AND (d_month_seq#27 >= 1200)) AND (d_month_seq#27 <= 1211)) AND isnotnull(d_date_sk#5)) +(28) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(26) CometProject -Input [2]: [d_date_sk#5, d_month_seq#27] +(29) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(27) ColumnarToRow [codegen id : 1] +(30) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(28) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt index be9c20a560..24e6dceef3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (8) +WholeStageCodegen (6) HashAggregate [sum,sum,sum] [sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),store_only,catalog_only,store_and_catalog,sum,sum,sum] InputAdapter Exchange #1 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [customer_sk,customer_sk] [sum,sum,sum,sum,sum,sum] Project [customer_sk,customer_sk] SortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] InputAdapter - WholeStageCodegen (3) + WholeStageCodegen (2) Sort [customer_sk,item_sk] HashAggregate [ss_customer_sk,ss_item_sk] [customer_sk,item_sk] InputAdapter Exchange [ss_customer_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_customer_sk,ss_item_sk] + CometProject [ss_item_sk,ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 @@ -27,21 +27,22 @@ WholeStageCodegen (8) CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (4) Sort [customer_sk,item_sk] HashAggregate [cs_bill_customer_sk,cs_item_sk] [customer_sk,item_sk] InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #4 - WholeStageCodegen (5) - HashAggregate [cs_bill_customer_sk,cs_item_sk] - Project [cs_bill_customer_sk,cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [cs_bill_customer_sk,cs_item_sk] + CometProject [cs_bill_customer_sk,cs_item_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index 3d66a07d07..ab7a40acdd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -1,26 +1,28 @@ == Physical Plan == -* Project (22) -+- * Sort (21) - +- Exchange (20) - +- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +* Project (24) ++- * Sort (23) + +- Exchange (22) + +- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * ColumnarToRow (15) + +- CometHashAggregate (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.store_sales @@ -35,126 +37,135 @@ ReadSchema: struct Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 27] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [ss_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) CometHashAggregate Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(15) ColumnarToRow [codegen id : 1] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] -(16) Exchange +(18) Exchange Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] +(19) Sort [codegen id : 3] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window +(20) Window Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(19) Project [codegen id : 6] +(21) Project [codegen id : 4] Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] -(20) Exchange +(22) Exchange Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] -Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) Sort [codegen id : 7] +(23) Sort [codegen id : 5] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], true, 0 -(22) Project [codegen id : 7] +(24) Project [codegen id : 5] Output [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) -(23) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(26) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(25) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(27) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index 9eabb9977c..c03b8be9a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -1,28 +1,28 @@ -WholeStageCodegen (7) +WholeStageCodegen (5) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] InputAdapter Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #2 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -33,12 +33,10 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #5 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt index 9365550266..1316871122 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt @@ -1,36 +1,32 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * Project (28) - +- * BroadcastHashJoin Inner BuildRight (27) - :- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.warehouse (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.ship_mode (10) - : +- BroadcastExchange (19) - : +- * ColumnarToRow (18) - : +- CometFilter (17) - : +- CometScan parquet spark_catalog.default.call_center (16) - +- BroadcastExchange (26) - +- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +TakeOrderedAndProject (28) ++- * HashAggregate (27) + +- Exchange (26) + +- * ColumnarToRow (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometProject (17) + : +- CometBroadcastHashJoin (16) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.warehouse (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.ship_mode (8) + : +- CometBroadcastExchange (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.call_center (13) + +- CometBroadcastExchange (21) + +- CometProject (20) + +- CometFilter (19) + +- CometScan parquet spark_catalog.default.date_dim (18) (1) Scan parquet spark_catalog.default.catalog_sales @@ -44,144 +40,126 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Condition : isnotnull(w_warehouse_sk#6) -(6) ColumnarToRow [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [w_warehouse_sk#6, w_warehouse_name#7] -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_warehouse_sk#4] -Right keys [1]: [w_warehouse_sk#6] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] +Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [cs_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight -(9) Project [codegen id : 5] -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] +(7) CometProject Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] +Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] -(10) Scan parquet spark_catalog.default.ship_mode +(8) Scan parquet spark_catalog.default.ship_mode Output [2]: [sm_ship_mode_sk#8, sm_type#9] Batched: true Location [not included in comparison]/{warehouse_dir}/ship_mode] PushedFilters: [IsNotNull(sm_ship_mode_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [sm_ship_mode_sk#8, sm_type#9] Condition : isnotnull(sm_ship_mode_sk#8) -(12) ColumnarToRow [codegen id : 2] -Input [2]: [sm_ship_mode_sk#8, sm_type#9] - -(13) BroadcastExchange +(10) CometBroadcastExchange Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +Arguments: [sm_ship_mode_sk#8, sm_type#9] -(14) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_ship_mode_sk#3] -Right keys [1]: [sm_ship_mode_sk#8] -Join type: Inner -Join condition: None +(11) CometBroadcastHashJoin +Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] +Right output [2]: [sm_ship_mode_sk#8, sm_type#9] +Arguments: [cs_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight -(15) Project [codegen id : 5] -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9] +(12) CometProject Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#9] +Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9] -(16) Scan parquet spark_catalog.default.call_center +(13) Scan parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#10, cc_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_call_center_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [2]: [cc_call_center_sk#10, cc_name#11] Condition : isnotnull(cc_call_center_sk#10) -(18) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [2]: [cc_call_center_sk#10, cc_name#11] +Arguments: [cc_call_center_sk#10, cc_name#11] -(19) BroadcastExchange -Input [2]: [cc_call_center_sk#10, cc_name#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_call_center_sk#2] -Right keys [1]: [cc_call_center_sk#10] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9] +Right output [2]: [cc_call_center_sk#10, cc_name#11] +Arguments: [cs_call_center_sk#2], [cc_call_center_sk#10], Inner, BuildRight -(21) Project [codegen id : 5] -Output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11] +(17) CometProject Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_call_center_sk#10, cc_name#11] +Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11], [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11] -(22) Scan parquet spark_catalog.default.date_dim +(18) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#12, d_month_seq#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(23) CometFilter +(19) CometFilter Input [2]: [d_date_sk#12, d_month_seq#13] Condition : (((isnotnull(d_month_seq#13) AND (d_month_seq#13 >= 1200)) AND (d_month_seq#13 <= 1211)) AND isnotnull(d_date_sk#12)) -(24) CometProject +(20) CometProject Input [2]: [d_date_sk#12, d_month_seq#13] Arguments: [d_date_sk#12], [d_date_sk#12] -(25) ColumnarToRow [codegen id : 4] +(21) CometBroadcastExchange Input [1]: [d_date_sk#12] +Arguments: [d_date_sk#12] -(26) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(22) CometBroadcastHashJoin +Left output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11] +Right output [1]: [d_date_sk#12] +Arguments: [cs_ship_date_sk#1], [d_date_sk#12], Inner, BuildRight -(27) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 5] -Output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] +(23) CometProject Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11, d_date_sk#12] +Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupingexpression#14], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] -(29) HashAggregate [codegen id : 5] +(24) CometHashAggregate Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupingexpression#14] Keys [3]: [_groupingexpression#14, sm_type#9, cc_name#11] Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] -Results [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -(30) Exchange -Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(25) ColumnarToRow [codegen id : 1] +Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#15, sum#16, sum#17, sum#18, sum#19] + +(26) Exchange +Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#15, sum#16, sum#17, sum#18, sum#19] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(31) HashAggregate [codegen id : 6] -Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] +(27) HashAggregate [codegen id : 2] +Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#15, sum#16, sum#17, sum#18, sum#19] Keys [3]: [_groupingexpression#14, sm_type#9, cc_name#11] Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] -Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] +Aggregate Attributes [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#20, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#21, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#22, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#23, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#24] +Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#25, sm_type#9, cc_name#11, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#20 AS 30 days #26, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#21 AS 31 - 60 days #27, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#22 AS 61 - 90 days #28, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#23 AS 91 - 120 days #29, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#24 AS >120 days #30] -(32) TakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] -Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] +(28) TakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#25, sm_type#9, cc_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] +Arguments: 100, [substr(w_warehouse_name, 1, 20)#25 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#25, sm_type#9, cc_name#11, 30 days #26, 31 - 60 days #27, 61 - 90 days #28, 91 - 120 days #29, >120 days #30] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt index c5f25f0795..adfe90babe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt @@ -1,48 +1,32 @@ TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] InputAdapter Exchange [_groupingexpression,sm_type,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,w_warehouse_name] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] + CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] + CometBroadcastHashJoin [cs_ship_date_sk,d_date_sk] + CometProject [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] + CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [sm_ship_mode_sk] - CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #2 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #3 + CometFilter [sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange #4 + CometFilter [cc_call_center_sk] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt index be0e98db20..43d59eb84f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt @@ -1,45 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * ColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (35) - +- * ColumnarToRow (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.customer_demographics (32) +TakeOrderedAndProject (39) ++- * HashAggregate (38) + +- Exchange (37) + +- * ColumnarToRow (36) + +- CometHashAggregate (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (29) + : +- CometBroadcastHashJoin (28) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometBroadcastHashJoin (11) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : +- CometBroadcastExchange (10) + : : : +- CometProject (9) + : : : +- CometBroadcastHashJoin (8) + : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : +- CometBroadcastExchange (21) + : : +- CometUnion (20) + : : :- CometProject (15) + : : : +- CometBroadcastHashJoin (14) + : : : :- CometScan parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (13) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (17) + : +- CometBroadcastExchange (27) + : +- CometProject (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer_address (24) + +- CometBroadcastExchange (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_demographics (30) (1) Scan parquet spark_catalog.default.customer @@ -53,220 +51,214 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] +ReadSchema: struct + +(5) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) -(6) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#7] +(6) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Arguments: [d_date_sk#7], [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight + +(9) CometProject Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#4] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ss_customer_sk#4] +Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(12) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#11] +(13) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customer_sk#12] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] +(15) CometProject +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +Arguments: [customer_sk#14], [ws_bill_customer_sk#10 AS customer_sk#14] (16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(17) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#18] -(18) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#16] +(18) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Right output [1]: [d_date_sk#18] +Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(19) CometProject +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] +Arguments: [customer_sk#19], [cs_ship_customer_sk#15 AS customer_sk#19] -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customer_sk#17] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] +(20) CometUnion +Child 0 Input [1]: [customer_sk#14] +Child 1 Input [1]: [customer_sk#19] -(21) Union - -(22) BroadcastExchange -Input [1]: [customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(21) CometBroadcastExchange +Input [1]: [customer_sk#14] +Arguments: [customer_sk#14] -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#12] -Join type: LeftSemi -Join condition: None +(22) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [customer_sk#14] +Arguments: [c_customer_sk#1], [customer_sk#14], LeftSemi, BuildRight -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +(23) CometProject Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(25) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_county#19] +(24) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(26) CometFilter -Input [2]: [ca_address_sk#18, ca_county#19] -Condition : (ca_county#19 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#18)) +(25) CometFilter +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) -(27) CometProject -Input [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] +(26) CometProject +Input [2]: [ca_address_sk#20, ca_county#21] +Arguments: [ca_address_sk#20], [ca_address_sk#20] -(28) ColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#18] +(27) CometBroadcastExchange +Input [1]: [ca_address_sk#20] +Arguments: [ca_address_sk#20] -(29) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(28) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ca_address_sk#20] +Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None +(29) CometProject +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] -(31) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] - -(32) Scan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(30) Scan parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(33) CometFilter -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#20) - -(34) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(31) CometFilter +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#22) -(35) BroadcastExchange -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(32) CometBroadcastExchange +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(36) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(33) CometBroadcastHashJoin +Left output [1]: [c_current_cdemo_sk#2] +Right output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight -(37) Project [codegen id : 9] -Output [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(34) CometProject +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(38) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(35) CometHashAggregate +Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -(39) Exchange -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(36) ColumnarToRow [codegen id : 1] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#31] -(40) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(37) Exchange +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#31] +Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(38) HashAggregate [codegen id : 2] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#31] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] +Aggregate Attributes [1]: [count(1)#32] +Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#32 AS cnt1#33, cd_purchase_estimate#26, count(1)#32 AS cnt2#34, cd_credit_rating#27, count(1)#32 AS cnt3#35, cd_dep_count#28, count(1)#32 AS cnt4#36, cd_dep_employed_count#29, count(1)#32 AS cnt5#37, cd_dep_college_count#30, count(1)#32 AS cnt6#38] -(41) TakeOrderedAndProject -Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] -Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] +(39) TakeOrderedAndProject +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#33, cd_purchase_estimate#26, cnt2#34, cd_credit_rating#27, cnt3#35, cd_dep_count#28, cnt4#36, cd_dep_employed_count#29, cnt5#37, cd_dep_college_count#30, cnt6#38] +Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#33, cd_purchase_estimate#26, cnt2#34, cd_credit_rating#27, cnt3#35, cd_dep_count#28, cnt4#36, cd_dep_employed_count#29, cnt5#37, cd_dep_college_count#30, cnt6#38] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (46) -+- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.date_dim (42) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) -(42) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#38, d_moy#39] +(40) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter -Input [3]: [d_date_sk#7, d_year#38, d_moy#39] -Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 4)) AND (d_moy#39 <= 7)) AND isnotnull(d_date_sk#7)) +(41) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) -(44) CometProject -Input [3]: [d_date_sk#7, d_year#38, d_moy#39] +(42) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(45) ColumnarToRow [codegen id : 1] +(43) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(46) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt index 3eb2210a6e..520edc88df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt @@ -1,72 +1,53 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) + WholeStageCodegen (2) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [c_current_cdemo_sk] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometUnion + CometProject [ws_bill_customer_sk] [customer_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometProject [cs_ship_customer_sk] [customer_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #6 + CometProject [ca_address_sk] + CometFilter [ca_county,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange #7 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt index daa1f52436..2b4fa79cd4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt @@ -2,74 +2,74 @@ TakeOrderedAndProject (71) +- * Project (70) +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * ColumnarToRow (36) - : : : +- CometFilter (35) - : : : +- CometScan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (17) + : : : +- * HashAggregate (16) + : : : +- Exchange (15) + : : : +- * HashAggregate (14) + : : : +- * ColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- BroadcastExchange (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * HashAggregate (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- CometBroadcastExchange (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- BroadcastExchange (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * HashAggregate (47) + : +- * ColumnarToRow (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometProject (42) + : : +- CometBroadcastHashJoin (41) + : : :- CometFilter (37) + : : : +- CometScan parquet spark_catalog.default.customer (36) + : : +- CometBroadcastExchange (40) + : : +- CometFilter (39) + : : +- CometScan parquet spark_catalog.default.web_sales (38) + : +- ReusedExchange (43) +- BroadcastExchange (68) +- * HashAggregate (67) +- Exchange (66) +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * ColumnarToRow (55) - : : +- CometFilter (54) - : : +- CometScan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) + +- * ColumnarToRow (64) + +- CometProject (63) + +- CometBroadcastHashJoin (62) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (61) (1) Scan parquet spark_catalog.default.customer @@ -83,10 +83,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] Condition : isnotnull(ss_customer_sk#9) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Right output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: [c_customer_sk#1], [ss_customer_sk#9], Inner, BuildRight -(9) Project [codegen id : 3] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +(7) CometProject Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(10) ReusedExchange [Reuses operator id: 75] +(8) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_year#15] +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 -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) + +(10) CometBroadcastExchange +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14, d_year#15] -(12) Project [codegen id : 3] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] +(11) CometBroadcastHashJoin +Left output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Right output [2]: [d_date_sk#14, d_year#15] +Arguments: [ss_sold_date_sk#12], [d_date_sk#14], Inner, BuildRight + +(12) CometProject Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#14, d_year#15] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] + +(13) ColumnarToRow [codegen id : 1] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] -(13) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] Aggregate Attributes [1]: [sum#16] Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -(14) Exchange +(15) Exchange Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 16] +(16) HashAggregate [codegen id : 8] Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] -(16) Filter [codegen id : 16] +(17) Filter [codegen id : 8] Input [2]: [customer_id#19, year_total#20] Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) -(17) Scan parquet spark_catalog.default.customer +(18) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter +(19) CometFilter Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) -(19) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] - (20) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] Batched: true @@ -176,79 +180,86 @@ ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(26) CometFilter +Input [2]: [d_date_sk#34, d_year#35] +Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) + +(27) CometBroadcastExchange +Input [2]: [d_date_sk#34, d_year#35] +Arguments: [d_date_sk#34, d_year#35] -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] +(28) CometBroadcastHashJoin +Left output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Right output [2]: [d_date_sk#34, d_year#35] +Arguments: [ss_sold_date_sk#32], [d_date_sk#34], Inner, BuildRight + +(29) CometProject Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] +Arguments: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35], [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] + +(30) ColumnarToRow [codegen id : 2] +Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] -(29) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] Aggregate Attributes [1]: [sum#36] Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -(30) Exchange +(32) Exchange Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(31) HashAggregate [codegen id : 7] +(33) HashAggregate [codegen id : 3] Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] Results [5]: [c_customer_id#22 AS customer_id#38, c_first_name#23 AS customer_first_name#39, c_last_name#24 AS customer_last_name#40, c_email_address#28 AS customer_email_address#41, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#42] -(32) BroadcastExchange +(34) BroadcastExchange Input [5]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 16] +(35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#38] Join type: Inner Join condition: None -(34) Scan parquet spark_catalog.default.customer +(36) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter +(37) CometFilter Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] Condition : (isnotnull(c_customer_sk#43) AND isnotnull(c_customer_id#44)) -(36) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] - -(37) Scan parquet spark_catalog.default.web_sales +(38) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] @@ -256,90 +267,85 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(38) CometFilter +(39) CometFilter Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Condition : isnotnull(ws_bill_customer_sk#51) -(39) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] - -(40) BroadcastExchange +(40) CometBroadcastExchange Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#43] -Right keys [1]: [ws_bill_customer_sk#51] -Join type: Inner -Join condition: None +(41) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] +Right output [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Arguments: [c_customer_sk#43], [ws_bill_customer_sk#51], Inner, BuildRight -(42) Project [codegen id : 10] -Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +(42) CometProject Input [12]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Arguments: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54], [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -(43) ReusedExchange [Reuses operator id: 75] +(43) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#56, d_year#57] -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#54] -Right keys [1]: [d_date_sk#56] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Right output [2]: [d_date_sk#56, d_year#57] +Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight -(45) Project [codegen id : 10] -Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] +(45) CometProject Input [12]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] +Arguments: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57], [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] -(46) HashAggregate [codegen id : 10] +(46) ColumnarToRow [codegen id : 4] +Input [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] + +(47) HashAggregate [codegen id : 4] Input [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] Aggregate Attributes [1]: [sum#58] Results [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] -(47) Exchange +(48) Exchange Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] -Arguments: hashpartitioning(c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(48) HashAggregate [codegen id : 11] +(49) HashAggregate [codegen id : 5] Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60] Results [2]: [c_customer_id#44 AS customer_id#61, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60,18,2) AS year_total#62] -(49) Filter [codegen id : 11] +(50) Filter [codegen id : 5] Input [2]: [customer_id#61, year_total#62] Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.00)) -(50) BroadcastExchange +(51) BroadcastExchange Input [2]: [customer_id#61, year_total#62] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(51) BroadcastHashJoin [codegen id : 16] +(52) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#61] Join type: Inner Join condition: None -(52) Project [codegen id : 16] +(53) Project [codegen id : 8] Output [8]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62] Input [9]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, customer_id#61, year_total#62] -(53) Scan parquet spark_catalog.default.customer +(54) Scan parquet spark_catalog.default.customer Output [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter +(55) CometFilter Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_customer_id#64)) -(55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] - (56) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] Batched: true @@ -352,37 +358,35 @@ ReadSchema: struct 0.00) THEN (year_total#81 / year_total#62) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END) -(70) Project [codegen id : 16] +(70) Project [codegen id : 8] Output [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] Input [10]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62, customer_id#80, year_total#81] @@ -420,7 +424,7 @@ Arguments: 100, [customer_id#38 ASC NULLS FIRST, customer_first_name#39 ASC NULL ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (75) +- * ColumnarToRow (74) +- CometFilter (73) @@ -443,7 +447,7 @@ Input [2]: [d_date_sk#14, d_year#15] (75) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 BroadcastExchange (79) @@ -468,9 +472,9 @@ Input [2]: [d_date_sk#34, d_year#35] (79) BroadcastExchange Input [2]: [d_date_sk#34, d_year#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#13 Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#33 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt index 0a30aba051..4b9e60e18f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - WholeStageCodegen (16) + WholeStageCodegen (8) Project [customer_id,customer_first_name,customer_last_name,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] @@ -9,114 +9,98 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + WholeStageCodegen (2) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange #7 CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #13 + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt index c39a71879d..9160db4e6b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * HashAggregate (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.web_sales @@ -33,118 +35,129 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [ws_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] +Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(15) HashAggregate [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Aggregate Attributes [1]: [sum#13] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] -(14) Exchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +(17) HashAggregate [codegen id : 2] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] -Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#15] +Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#15,17,2) AS itemrevenue#16, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#15,17,2) AS _w0#17] -(16) Exchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(18) Exchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +(19) Sort [codegen id : 3] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(20) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] +Arguments: [sum(_w0#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9] -(19) Project [codegen id : 6] -Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] -Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] +(21) Project [codegen id : 4] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, ((_w0#17 * 100) / _we0#18) AS revenueratio#19] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, _we0#18] -(20) TakeOrderedAndProject -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +(22) TakeOrderedAndProject +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#19 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) -(21) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(23) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt index 1bc2538b48..3d05766538 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,12 +29,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt index ec52cea9f9..ca217cde10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt @@ -23,43 +23,43 @@ TakeOrderedAndProject (84) : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) : : : :- * HashAggregate (35) : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) + : : : : +- * ColumnarToRow (33) + : : : : +- CometHashAggregate (32) + : : : : +- CometProject (31) + : : : : +- CometBroadcastHashJoin (30) + : : : : :- CometProject (28) + : : : : : +- CometBroadcastHashJoin (27) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- CometBroadcastExchange (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (10) + : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometProject (23) + : : : : : +- CometBroadcastHashJoin (22) + : : : : : :- CometProject (17) + : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : :- CometFilter (12) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : +- CometBroadcastExchange (15) + : : : : : : +- CometFilter (14) + : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : +- CometBroadcastExchange (21) + : : : : : +- CometProject (20) + : : : : : +- CometFilter (19) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) + : : : : +- ReusedExchange (29) : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * ColumnarToRow (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) + : : : +- * ColumnarToRow (44) + : : : +- CometProject (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (38) + : : : +- ReusedExchange (41) : : +- BroadcastExchange (57) : : +- * BroadcastHashJoin LeftSemi BuildRight (56) : : :- * ColumnarToRow (54) @@ -97,7 +97,7 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 25] +(3) ColumnarToRow [codegen id : 11] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (4) Scan parquet spark_catalog.default.item @@ -111,7 +111,7 @@ ReadSchema: struct Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -151,609 +145,605 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) ColumnarToRow [codegen id : 1] +(15) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(19) BroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(17) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#24] +(18) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(19) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) + +(20) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] + +(21) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(22) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight + +(23) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(24) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(25) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(28) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#25] +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(31) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] (34) Exchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] +Results [3]: [brand_id#27, class_id#28, category_id#29] (36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (37) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(42) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#36] +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] (45) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(48) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(49) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] (50) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [ss_item_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (52) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (53) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) -(54) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] (55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (57) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(58) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(59) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#42] +(60) ReusedExchange [Reuses operator id: 106] +Output [1]: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(62) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(63) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] (64) Exchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(65) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(65) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [6]: [store AS channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#53, count(1)#51 AS number_sales#54] -(66) Filter [codegen id : 52] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(66) Filter [codegen id : 24] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Condition : (isnotnull(sales#53) AND (cast(sales#53 as decimal(32,6)) > cast(Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (67) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (68) CometFilter -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] -Condition : isnotnull(ss_item_sk#56) +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +Condition : isnotnull(ss_item_sk#57) -(69) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +(69) ColumnarToRow [codegen id : 22] +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] (70) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#61] +Output [1]: [ss_item_sk#62] -(71) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [ss_item_sk#61] +(71) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [ss_item_sk#62] Join type: LeftSemi Join condition: None (72) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] +Output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [i_item_sk#62] +(73) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [i_item_sk#63] Join type: Inner Join condition: None -(74) Project [codegen id : 50] -Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] +(74) Project [codegen id : 22] +Output [6]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] +Input [8]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -(75) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#66] +(75) ReusedExchange [Reuses operator id: 120] +Output [1]: [d_date_sk#67] -(76) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#66] +(76) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_sold_date_sk#60] +Right keys [1]: [d_date_sk#67] Join type: Inner Join condition: None -(77) Project [codegen id : 50] -Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] +(77) Project [codegen id : 22] +Output [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] +Input [7]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66, d_date_sk#67] -(78) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +(78) HashAggregate [codegen id : 22] +Input [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] +Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] +Functions [2]: [partial_sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), partial_count(1)] +Aggregate Attributes [3]: [sum#68, isEmpty#69, count#70] +Results [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73] (79) Exchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73] +Arguments: hashpartitioning(i_brand_id#64, i_class_id#65, i_category_id#66, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(80) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73, count(1)#74] -Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] +(80) HashAggregate [codegen id : 23] +Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73] +Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] +Functions [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#74, count(1)#75] +Results [6]: [store AS channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#74 AS sales#77, count(1)#75 AS number_sales#78] -(81) Filter [codegen id : 51] -Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(81) Filter [codegen id : 23] +Input [6]: [channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78] +Condition : (isnotnull(sales#77) AND (cast(sales#77 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (82) BroadcastExchange -Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] +Input [6]: [channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=8] -(83) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +(83) BroadcastHashJoin [codegen id : 24] +Left keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +Right keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] Join type: Inner Join condition: None (84) TakeOrderedAndProject -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Input [12]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78] +Arguments: 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (103) -+- Exchange (102) - +- * HashAggregate (101) - +- Union (100) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * ColumnarToRow (86) - : : +- CometScan parquet spark_catalog.default.store_sales (85) - : +- ReusedExchange (87) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * ColumnarToRow (91) - : : +- CometScan parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (92) - +- * Project (99) - +- * BroadcastHashJoin Inner BuildRight (98) - :- * ColumnarToRow (96) - : +- CometScan parquet spark_catalog.default.web_sales (95) - +- ReusedExchange (97) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#55, [id=#56] +* HashAggregate (101) ++- Exchange (100) + +- * HashAggregate (99) + +- * ColumnarToRow (98) + +- CometUnion (97) + :- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometScan parquet spark_catalog.default.store_sales (85) + : +- ReusedExchange (86) + :- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometScan parquet spark_catalog.default.catalog_sales (89) + : +- ReusedExchange (90) + +- CometProject (96) + +- CometBroadcastHashJoin (95) + :- CometScan parquet spark_catalog.default.web_sales (93) + +- ReusedExchange (94) (85) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] +Output [3]: [ss_quantity#79, ss_list_price#80, ss_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] +PartitionFilters: [isnotnull(ss_sold_date_sk#81), dynamicpruningexpression(ss_sold_date_sk#81 IN dynamicpruning#82)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] - -(87) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#82] +(86) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#83] -(88) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#80] -Right keys [1]: [d_date_sk#82] -Join type: Inner -Join condition: None +(87) CometBroadcastHashJoin +Left output [3]: [ss_quantity#79, ss_list_price#80, ss_sold_date_sk#81] +Right output [1]: [d_date_sk#83] +Arguments: [ss_sold_date_sk#81], [d_date_sk#83], Inner, BuildRight -(89) Project [codegen id : 2] -Output [2]: [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] -Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] +(88) CometProject +Input [4]: [ss_quantity#79, ss_list_price#80, ss_sold_date_sk#81, d_date_sk#83] +Arguments: [quantity#84, list_price#85], [ss_quantity#79 AS quantity#84, ss_list_price#80 AS list_price#85] -(90) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +(89) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] +PartitionFilters: [isnotnull(cs_sold_date_sk#88), dynamicpruningexpression(cs_sold_date_sk#88 IN dynamicpruning#89)] ReadSchema: struct -(91) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +(90) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#90] -(92) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#89] +(91) CometBroadcastHashJoin +Left output [3]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88] +Right output [1]: [d_date_sk#90] +Arguments: [cs_sold_date_sk#88], [d_date_sk#90], Inner, BuildRight -(93) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#87] -Right keys [1]: [d_date_sk#89] -Join type: Inner -Join condition: None - -(94) Project [codegen id : 4] -Output [2]: [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] -Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] +(92) CometProject +Input [4]: [cs_quantity#86, cs_list_price#87, cs_sold_date_sk#88, d_date_sk#90] +Arguments: [quantity#91, list_price#92], [cs_quantity#86 AS quantity#91, cs_list_price#87 AS list_price#92] -(95) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(93) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] +PartitionFilters: [isnotnull(ws_sold_date_sk#95), dynamicpruningexpression(ws_sold_date_sk#95 IN dynamicpruning#96)] ReadSchema: struct -(96) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +(94) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#97] -(97) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#96] +(95) CometBroadcastHashJoin +Left output [3]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95] +Right output [1]: [d_date_sk#97] +Arguments: [ws_sold_date_sk#95], [d_date_sk#97], Inner, BuildRight -(98) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#94] -Right keys [1]: [d_date_sk#96] -Join type: Inner -Join condition: None +(96) CometProject +Input [4]: [ws_quantity#93, ws_list_price#94, ws_sold_date_sk#95, d_date_sk#97] +Arguments: [quantity#98, list_price#99], [ws_quantity#93 AS quantity#98, ws_list_price#94 AS list_price#99] -(99) Project [codegen id : 6] -Output [2]: [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] -Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] +(97) CometUnion +Child 0 Input [2]: [quantity#84, list_price#85] +Child 1 Input [2]: [quantity#91, list_price#92] +Child 2 Input [2]: [quantity#98, list_price#99] -(100) Union +(98) ColumnarToRow [codegen id : 1] +Input [2]: [quantity#84, list_price#85] -(101) HashAggregate [codegen id : 7] -Input [2]: [quantity#83, list_price#84] +(99) HashAggregate [codegen id : 1] +Input [2]: [quantity#84, list_price#85] Keys: [] -Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] -Aggregate Attributes [2]: [sum#99, count#100] -Results [2]: [sum#101, count#102] +Functions [1]: [partial_avg((cast(quantity#84 as decimal(10,0)) * list_price#85))] +Aggregate Attributes [2]: [sum#100, count#101] +Results [2]: [sum#102, count#103] -(102) Exchange -Input [2]: [sum#101, count#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(100) Exchange +Input [2]: [sum#102, count#103] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(103) HashAggregate [codegen id : 8] -Input [2]: [sum#101, count#102] +(101) HashAggregate [codegen id : 2] +Input [2]: [sum#102, count#103] Keys: [] -Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] -Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103] -Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103 AS average_sales#104] +Functions [1]: [avg((cast(quantity#84 as decimal(10,0)) * list_price#85))] +Aggregate Attributes [1]: [avg((cast(quantity#84 as decimal(10,0)) * list_price#85))#104] +Results [1]: [avg((cast(quantity#84 as decimal(10,0)) * list_price#85))#104 AS average_sales#105] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#81 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#88 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#95 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * ColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan parquet spark_catalog.default.date_dim (104) +BroadcastExchange (106) ++- * ColumnarToRow (105) + +- CometProject (104) + +- CometFilter (103) + +- CometScan parquet spark_catalog.default.date_dim (102) -(104) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#42, d_week_seq#105] +(102) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#106] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#106), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#107), IsNotNull(d_date_sk)] ReadSchema: struct -(105) CometFilter -Input [2]: [d_date_sk#42, d_week_seq#105] -Condition : ((isnotnull(d_week_seq#105) AND (d_week_seq#105 = ReusedSubquery Subquery scalar-subquery#106, [id=#107])) AND isnotnull(d_date_sk#42)) +(103) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#106] +Condition : ((isnotnull(d_week_seq#106) AND (d_week_seq#106 = ReusedSubquery Subquery scalar-subquery#107, [id=#108])) AND isnotnull(d_date_sk#43)) -(106) CometProject -Input [2]: [d_date_sk#42, d_week_seq#105] -Arguments: [d_date_sk#42], [d_date_sk#42] +(104) CometProject +Input [2]: [d_date_sk#43, d_week_seq#106] +Arguments: [d_date_sk#43], [d_date_sk#43] -(107) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(105) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(108) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(106) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:6 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#106, [id=#107] +Subquery:6 Hosting operator id = 103 Hosting Expression = ReusedSubquery Subquery scalar-subquery#107, [id=#108] -Subquery:7 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#106, [id=#107] -* ColumnarToRow (112) -+- CometProject (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +Subquery:7 Hosting operator id = 102 Hosting Expression = Subquery scalar-subquery#107, [id=#108] +* ColumnarToRow (110) ++- CometProject (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(109) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +(107) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#109, d_year#110, d_moy#111, d_dom#112] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(110) CometFilter -Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] -Condition : (((((isnotnull(d_year#109) AND isnotnull(d_moy#110)) AND isnotnull(d_dom#111)) AND (d_year#109 = 1999)) AND (d_moy#110 = 12)) AND (d_dom#111 = 16)) +(108) CometFilter +Input [4]: [d_week_seq#109, d_year#110, d_moy#111, d_dom#112] +Condition : (((((isnotnull(d_year#110) AND isnotnull(d_moy#111)) AND isnotnull(d_dom#112)) AND (d_year#110 = 1999)) AND (d_moy#111 = 12)) AND (d_dom#112 = 16)) -(111) CometProject -Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] -Arguments: [d_week_seq#108], [d_week_seq#108] +(109) CometProject +Input [4]: [d_week_seq#109, d_year#110, d_moy#111, d_dom#112] +Arguments: [d_week_seq#109], [d_week_seq#109] -(112) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#108] +(110) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#109] Subquery:8 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (115) ++- * ColumnarToRow (114) + +- CometProject (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) -(113) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#112] +(111) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#113] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter -Input [2]: [d_date_sk#25, d_year#112] -Condition : (((isnotnull(d_year#112) AND (d_year#112 >= 1998)) AND (d_year#112 <= 2000)) AND isnotnull(d_date_sk#25)) +(112) CometFilter +Input [2]: [d_date_sk#26, d_year#113] +Condition : (((isnotnull(d_year#113) AND (d_year#113 >= 1998)) AND (d_year#113 <= 2000)) AND isnotnull(d_date_sk#26)) -(115) CometProject -Input [2]: [d_date_sk#25, d_year#112] -Arguments: [d_date_sk#25], [d_date_sk#25] +(113) CometProject +Input [2]: [d_date_sk#26, d_year#113] +Arguments: [d_date_sk#26], [d_date_sk#26] -(116) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(114) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(117) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(115) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:9 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:10 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:12 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 -BroadcastExchange (122) -+- * ColumnarToRow (121) - +- CometProject (120) - +- CometFilter (119) - +- CometScan parquet spark_catalog.default.date_dim (118) +Subquery:12 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#60 IN dynamicpruning#61 +BroadcastExchange (120) ++- * ColumnarToRow (119) + +- CometProject (118) + +- CometFilter (117) + +- CometScan parquet spark_catalog.default.date_dim (116) -(118) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_week_seq#113] +(116) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#67, d_week_seq#114] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#114), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#115), IsNotNull(d_date_sk)] ReadSchema: struct -(119) CometFilter -Input [2]: [d_date_sk#66, d_week_seq#113] -Condition : ((isnotnull(d_week_seq#113) AND (d_week_seq#113 = ReusedSubquery Subquery scalar-subquery#114, [id=#115])) AND isnotnull(d_date_sk#66)) +(117) CometFilter +Input [2]: [d_date_sk#67, d_week_seq#114] +Condition : ((isnotnull(d_week_seq#114) AND (d_week_seq#114 = ReusedSubquery Subquery scalar-subquery#115, [id=#116])) AND isnotnull(d_date_sk#67)) -(120) CometProject -Input [2]: [d_date_sk#66, d_week_seq#113] -Arguments: [d_date_sk#66], [d_date_sk#66] +(118) CometProject +Input [2]: [d_date_sk#67, d_week_seq#114] +Arguments: [d_date_sk#67], [d_date_sk#67] -(121) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#66] +(119) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#67] -(122) BroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +(120) BroadcastExchange +Input [1]: [d_date_sk#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:13 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#114, [id=#115] +Subquery:13 Hosting operator id = 117 Hosting Expression = ReusedSubquery Subquery scalar-subquery#115, [id=#116] -Subquery:14 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#114, [id=#115] -* ColumnarToRow (126) -+- CometProject (125) - +- CometFilter (124) - +- CometScan parquet spark_catalog.default.date_dim (123) +Subquery:14 Hosting operator id = 116 Hosting Expression = Subquery scalar-subquery#115, [id=#116] +* ColumnarToRow (124) ++- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) -(123) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +(121) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(124) CometFilter -Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#117) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#117 = 1998)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) +(122) CometFilter +Input [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] +Condition : (((((isnotnull(d_year#118) AND isnotnull(d_moy#119)) AND isnotnull(d_dom#120)) AND (d_year#118 = 1998)) AND (d_moy#119 = 12)) AND (d_dom#120 = 16)) -(125) CometProject -Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] -Arguments: [d_week_seq#116], [d_week_seq#116] +(123) CometProject +Input [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] +Arguments: [d_week_seq#117], [d_week_seq#117] -(126) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#116] +(124) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#117] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt index 55aa823ab8..73badab8db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt @@ -1,47 +1,36 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (24) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #12 - WholeStageCodegen (7) + Exchange #13 + WholeStageCodegen (1) HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 + ColumnarToRow + InputAdapter + CometUnion + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (11) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -70,7 +59,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter BroadcastExchange #3 - WholeStageCodegen (11) + WholeStageCodegen (4) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow @@ -79,19 +68,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (10) + WholeStageCodegen (3) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 @@ -102,56 +91,43 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 + CometBroadcastExchange #7 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange #9 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #10 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #11 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) + BroadcastExchange #12 + WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter @@ -162,14 +138,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) + BroadcastExchange #14 + WholeStageCodegen (23) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (22) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -181,7 +157,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 + BroadcastExchange #16 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -199,6 +175,6 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter - ReusedExchange [d_date_sk] #15 + ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt index d76059e019..b0944131ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt @@ -30,43 +30,43 @@ TakeOrderedAndProject (125) : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) : : : : :- * HashAggregate (35) : : : : : +- Exchange (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * ColumnarToRow (9) - : : : : : : : +- CometFilter (8) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * ColumnarToRow (12) - : : : : : : : +- CometFilter (11) - : : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * ColumnarToRow (15) - : : : : : : : : +- CometFilter (14) - : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) + : : : : : +- * ColumnarToRow (33) + : : : : : +- CometHashAggregate (32) + : : : : : +- CometProject (31) + : : : : : +- CometBroadcastHashJoin (30) + : : : : : :- CometProject (28) + : : : : : : +- CometBroadcastHashJoin (27) + : : : : : : :- CometFilter (8) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : : +- CometBroadcastExchange (26) + : : : : : : +- CometBroadcastHashJoin (25) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : : +- CometBroadcastExchange (24) + : : : : : : +- CometProject (23) + : : : : : : +- CometBroadcastHashJoin (22) + : : : : : : :- CometProject (17) + : : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : : :- CometFilter (12) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : : +- CometBroadcastExchange (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : : +- CometBroadcastExchange (21) + : : : : : : +- CometProject (20) + : : : : : : +- CometFilter (19) + : : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) + : : : : : +- ReusedExchange (29) : : : : +- BroadcastExchange (45) - : : : : +- * Project (44) - : : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : : :- * Project (41) - : : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : : :- * ColumnarToRow (38) - : : : : : : +- CometFilter (37) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : : +- ReusedExchange (39) - : : : : +- ReusedExchange (42) + : : : : +- * ColumnarToRow (44) + : : : : +- CometProject (43) + : : : : +- CometBroadcastHashJoin (42) + : : : : :- CometProject (40) + : : : : : +- CometBroadcastHashJoin (39) + : : : : : :- CometFilter (37) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : : +- ReusedExchange (38) + : : : : +- ReusedExchange (41) : : : +- BroadcastExchange (57) : : : +- * BroadcastHashJoin LeftSemi BuildRight (56) : : : :- * ColumnarToRow (54) @@ -138,7 +138,7 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 25] +(3) ColumnarToRow [codegen id : 11] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (4) Scan parquet spark_catalog.default.item @@ -152,7 +152,7 @@ ReadSchema: struct Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -192,773 +186,788 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(19) BroadcastExchange +(15) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(17) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#24] +(18) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(19) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(20) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] + +(21) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] + +(22) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight + +(23) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(24) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(25) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(28) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#25] +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(31) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] (34) Exchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] +Results [3]: [brand_id#27, class_id#28, category_id#29] (36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (37) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(42) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#36] +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] (45) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(48) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(49) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] (50) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [ss_item_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (52) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (53) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : isnotnull(i_item_sk#38) +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : isnotnull(i_item_sk#39) -(54) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] (55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (57) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(58) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(59) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#42] +(60) ReusedExchange [Reuses operator id: 155] +Output [1]: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(62) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(63) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] (64) Exchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(65) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(65) HashAggregate [codegen id : 12] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [6]: [store AS channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#53, count(1)#51 AS number_sales#54] -(66) Filter [codegen id : 26] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(66) Filter [codegen id : 12] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Condition : (isnotnull(sales#53) AND (cast(sales#53 as decimal(32,6)) > cast(Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (67) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#59), dynamicpruningexpression(cs_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (68) CometFilter -Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] -Condition : isnotnull(cs_item_sk#56) +Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +Condition : isnotnull(cs_item_sk#57) -(69) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +(69) ColumnarToRow [codegen id : 23] +Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] (70) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#61] +Output [1]: [ss_item_sk#62] -(71) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#56] -Right keys [1]: [ss_item_sk#61] +(71) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#57] +Right keys [1]: [ss_item_sk#62] Join type: LeftSemi Join condition: None (72) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] +Output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -(73) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#56] -Right keys [1]: [i_item_sk#62] +(73) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#57] +Right keys [1]: [i_item_sk#63] Join type: Inner Join condition: None -(74) Project [codegen id : 51] -Output [6]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [8]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] +(74) Project [codegen id : 23] +Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] +Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -(75) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#66] +(75) ReusedExchange [Reuses operator id: 155] +Output [1]: [d_date_sk#67] -(76) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#59] -Right keys [1]: [d_date_sk#66] +(76) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_sold_date_sk#60] +Right keys [1]: [d_date_sk#67] Join type: Inner Join condition: None -(77) Project [codegen id : 51] -Output [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [7]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] +(77) Project [codegen id : 23] +Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] +Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66, d_date_sk#67] -(78) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +(78) HashAggregate [codegen id : 23] +Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] +Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] +Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] +Aggregate Attributes [3]: [sum#68, isEmpty#69, count#70] +Results [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73] (79) Exchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73] +Arguments: hashpartitioning(i_brand_id#64, i_class_id#65, i_category_id#66, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(80) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73, count(1)#74] -Results [6]: [catalog AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] +(80) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#71, isEmpty#72, count#73] +Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] +Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#74, count(1)#75] +Results [6]: [catalog AS channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#74 AS sales#77, count(1)#75 AS number_sales#78] -(81) Filter [codegen id : 52] -Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] -Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(81) Filter [codegen id : 24] +Input [6]: [channel#76, i_brand_id#64, i_class_id#65, i_category_id#66, sales#77, number_sales#78] +Condition : (isnotnull(sales#77) AND (cast(sales#77 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (82) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +Output [4]: [ws_item_sk#79, ws_quantity#80, ws_list_price#81, ws_sold_date_sk#82] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#82), dynamicpruningexpression(ws_sold_date_sk#82 IN dynamicpruning#83)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (83) CometFilter -Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_item_sk#78) +Input [4]: [ws_item_sk#79, ws_quantity#80, ws_list_price#81, ws_sold_date_sk#82] +Condition : isnotnull(ws_item_sk#79) -(84) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +(84) ColumnarToRow [codegen id : 35] +Input [4]: [ws_item_sk#79, ws_quantity#80, ws_list_price#81, ws_sold_date_sk#82] (85) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#83] +Output [1]: [ss_item_sk#84] -(86) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#78] -Right keys [1]: [ss_item_sk#83] +(86) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#79] +Right keys [1]: [ss_item_sk#84] Join type: LeftSemi Join condition: None (87) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] +Output [4]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] -(88) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#78] -Right keys [1]: [i_item_sk#84] +(88) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#79] +Right keys [1]: [i_item_sk#85] Join type: Inner Join condition: None -(89) Project [codegen id : 77] -Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] +(89) Project [codegen id : 35] +Output [6]: [ws_quantity#80, ws_list_price#81, ws_sold_date_sk#82, i_brand_id#86, i_class_id#87, i_category_id#88] +Input [8]: [ws_item_sk#79, ws_quantity#80, ws_list_price#81, ws_sold_date_sk#82, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88] -(90) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#88] +(90) ReusedExchange [Reuses operator id: 155] +Output [1]: [d_date_sk#89] -(91) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#88] +(91) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_sold_date_sk#82] +Right keys [1]: [d_date_sk#89] Join type: Inner Join condition: None -(92) Project [codegen id : 77] -Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] +(92) Project [codegen id : 35] +Output [5]: [ws_quantity#80, ws_list_price#81, i_brand_id#86, i_class_id#87, i_category_id#88] +Input [7]: [ws_quantity#80, ws_list_price#81, ws_sold_date_sk#82, i_brand_id#86, i_class_id#87, i_category_id#88, d_date_sk#89] -(93) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] -Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] -Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +(93) HashAggregate [codegen id : 35] +Input [5]: [ws_quantity#80, ws_list_price#81, i_brand_id#86, i_class_id#87, i_category_id#88] +Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] +Functions [2]: [partial_sum((cast(ws_quantity#80 as decimal(10,0)) * ws_list_price#81)), partial_count(1)] +Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] +Results [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] (94) Exchange -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(95) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95, count(1)#96] -Results [6]: [web AS channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95 AS sales#98, count(1)#96 AS number_sales#99] +(95) HashAggregate [codegen id : 36] +Input [6]: [i_brand_id#86, i_class_id#87, i_category_id#88, sum#93, isEmpty#94, count#95] +Keys [3]: [i_brand_id#86, i_class_id#87, i_category_id#88] +Functions [2]: [sum((cast(ws_quantity#80 as decimal(10,0)) * ws_list_price#81)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#80 as decimal(10,0)) * ws_list_price#81))#96, count(1)#97] +Results [6]: [web AS channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sum((cast(ws_quantity#80 as decimal(10,0)) * ws_list_price#81))#96 AS sales#99, count(1)#97 AS number_sales#100] -(96) Filter [codegen id : 78] -Input [6]: [channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99] -Condition : (isnotnull(sales#98) AND (cast(sales#98 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(96) Filter [codegen id : 36] +Input [6]: [channel#98, i_brand_id#86, i_class_id#87, i_category_id#88, sales#99, number_sales#100] +Condition : (isnotnull(sales#99) AND (cast(sales#99 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (97) Union -(98) HashAggregate [codegen id : 79] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] -Aggregate Attributes [3]: [sum#100, isEmpty#101, sum#102] -Results [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] +(98) HashAggregate [codegen id : 37] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum(sales#53), partial_sum(number_sales#54)] +Aggregate Attributes [3]: [sum#101, isEmpty#102, sum#103] +Results [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#104, isEmpty#105, sum#106] (99) Exchange -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#104, isEmpty#105, sum#106] +Arguments: hashpartitioning(channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(100) HashAggregate [codegen id : 80] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum(sales#52)#106 AS sum_sales#108, sum(number_sales#53)#107 AS number_sales#109] +(100) HashAggregate [codegen id : 38] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#104, isEmpty#105, sum#106] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#53), sum(number_sales#54)] +Aggregate Attributes [2]: [sum(sales#53)#107, sum(number_sales#54)#108] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum(sales#53)#107 AS sum_sales#109, sum(number_sales#54)#108 AS number_sales#110] (101) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, sum#114, isEmpty#115, sum#116] - -(102) HashAggregate [codegen id : 160] -Input [7]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, sum#114, isEmpty#115, sum#116] -Keys [4]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113] -Functions [2]: [sum(sales#117), sum(number_sales#118)] -Aggregate Attributes [2]: [sum(sales#117)#106, sum(number_sales#118)#107] -Results [5]: [channel#110, i_brand_id#111, i_class_id#112, sum(sales#117)#106 AS sum_sales#119, sum(number_sales#118)#107 AS number_sales#120] - -(103) HashAggregate [codegen id : 160] -Input [5]: [channel#110, i_brand_id#111, i_class_id#112, sum_sales#119, number_sales#120] -Keys [3]: [channel#110, i_brand_id#111, i_class_id#112] -Functions [2]: [partial_sum(sum_sales#119), partial_sum(number_sales#120)] -Aggregate Attributes [3]: [sum#121, isEmpty#122, sum#123] -Results [6]: [channel#110, i_brand_id#111, i_class_id#112, sum#124, isEmpty#125, sum#126] +Output [7]: [channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, sum#115, isEmpty#116, sum#117] + +(102) HashAggregate [codegen id : 76] +Input [7]: [channel#111, i_brand_id#112, i_class_id#113, i_category_id#114, sum#115, isEmpty#116, sum#117] +Keys [4]: [channel#111, i_brand_id#112, i_class_id#113, i_category_id#114] +Functions [2]: [sum(sales#118), sum(number_sales#119)] +Aggregate Attributes [2]: [sum(sales#118)#107, sum(number_sales#119)#108] +Results [5]: [channel#111, i_brand_id#112, i_class_id#113, sum(sales#118)#107 AS sum_sales#120, sum(number_sales#119)#108 AS number_sales#121] + +(103) HashAggregate [codegen id : 76] +Input [5]: [channel#111, i_brand_id#112, i_class_id#113, sum_sales#120, number_sales#121] +Keys [3]: [channel#111, i_brand_id#112, i_class_id#113] +Functions [2]: [partial_sum(sum_sales#120), partial_sum(number_sales#121)] +Aggregate Attributes [3]: [sum#122, isEmpty#123, sum#124] +Results [6]: [channel#111, i_brand_id#112, i_class_id#113, sum#125, isEmpty#126, sum#127] (104) Exchange -Input [6]: [channel#110, i_brand_id#111, i_class_id#112, sum#124, isEmpty#125, sum#126] -Arguments: hashpartitioning(channel#110, i_brand_id#111, i_class_id#112, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [6]: [channel#111, i_brand_id#112, i_class_id#113, sum#125, isEmpty#126, sum#127] +Arguments: hashpartitioning(channel#111, i_brand_id#112, i_class_id#113, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(105) HashAggregate [codegen id : 161] -Input [6]: [channel#110, i_brand_id#111, i_class_id#112, sum#124, isEmpty#125, sum#126] -Keys [3]: [channel#110, i_brand_id#111, i_class_id#112] -Functions [2]: [sum(sum_sales#119), sum(number_sales#120)] -Aggregate Attributes [2]: [sum(sum_sales#119)#127, sum(number_sales#120)#128] -Results [6]: [channel#110, i_brand_id#111, i_class_id#112, null AS i_category_id#129, sum(sum_sales#119)#127 AS sum(sum_sales)#130, sum(number_sales#120)#128 AS sum(number_sales)#131] +(105) HashAggregate [codegen id : 77] +Input [6]: [channel#111, i_brand_id#112, i_class_id#113, sum#125, isEmpty#126, sum#127] +Keys [3]: [channel#111, i_brand_id#112, i_class_id#113] +Functions [2]: [sum(sum_sales#120), sum(number_sales#121)] +Aggregate Attributes [2]: [sum(sum_sales#120)#128, sum(number_sales#121)#129] +Results [6]: [channel#111, i_brand_id#112, i_class_id#113, null AS i_category_id#130, sum(sum_sales#120)#128 AS sum(sum_sales)#131, sum(number_sales#121)#129 AS sum(number_sales)#132] (106) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#132, i_brand_id#133, i_class_id#134, i_category_id#135, sum#136, isEmpty#137, sum#138] - -(107) HashAggregate [codegen id : 241] -Input [7]: [channel#132, i_brand_id#133, i_class_id#134, i_category_id#135, sum#136, isEmpty#137, sum#138] -Keys [4]: [channel#132, i_brand_id#133, i_class_id#134, i_category_id#135] -Functions [2]: [sum(sales#139), sum(number_sales#140)] -Aggregate Attributes [2]: [sum(sales#139)#106, sum(number_sales#140)#107] -Results [4]: [channel#132, i_brand_id#133, sum(sales#139)#106 AS sum_sales#141, sum(number_sales#140)#107 AS number_sales#142] - -(108) HashAggregate [codegen id : 241] -Input [4]: [channel#132, i_brand_id#133, sum_sales#141, number_sales#142] -Keys [2]: [channel#132, i_brand_id#133] -Functions [2]: [partial_sum(sum_sales#141), partial_sum(number_sales#142)] -Aggregate Attributes [3]: [sum#143, isEmpty#144, sum#145] -Results [5]: [channel#132, i_brand_id#133, sum#146, isEmpty#147, sum#148] +Output [7]: [channel#133, i_brand_id#134, i_class_id#135, i_category_id#136, sum#137, isEmpty#138, sum#139] + +(107) HashAggregate [codegen id : 115] +Input [7]: [channel#133, i_brand_id#134, i_class_id#135, i_category_id#136, sum#137, isEmpty#138, sum#139] +Keys [4]: [channel#133, i_brand_id#134, i_class_id#135, i_category_id#136] +Functions [2]: [sum(sales#140), sum(number_sales#141)] +Aggregate Attributes [2]: [sum(sales#140)#107, sum(number_sales#141)#108] +Results [4]: [channel#133, i_brand_id#134, sum(sales#140)#107 AS sum_sales#142, sum(number_sales#141)#108 AS number_sales#143] + +(108) HashAggregate [codegen id : 115] +Input [4]: [channel#133, i_brand_id#134, sum_sales#142, number_sales#143] +Keys [2]: [channel#133, i_brand_id#134] +Functions [2]: [partial_sum(sum_sales#142), partial_sum(number_sales#143)] +Aggregate Attributes [3]: [sum#144, isEmpty#145, sum#146] +Results [5]: [channel#133, i_brand_id#134, sum#147, isEmpty#148, sum#149] (109) Exchange -Input [5]: [channel#132, i_brand_id#133, sum#146, isEmpty#147, sum#148] -Arguments: hashpartitioning(channel#132, i_brand_id#133, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Input [5]: [channel#133, i_brand_id#134, sum#147, isEmpty#148, sum#149] +Arguments: hashpartitioning(channel#133, i_brand_id#134, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(110) HashAggregate [codegen id : 242] -Input [5]: [channel#132, i_brand_id#133, sum#146, isEmpty#147, sum#148] -Keys [2]: [channel#132, i_brand_id#133] -Functions [2]: [sum(sum_sales#141), sum(number_sales#142)] -Aggregate Attributes [2]: [sum(sum_sales#141)#149, sum(number_sales#142)#150] -Results [6]: [channel#132, i_brand_id#133, null AS i_class_id#151, null AS i_category_id#152, sum(sum_sales#141)#149 AS sum(sum_sales)#153, sum(number_sales#142)#150 AS sum(number_sales)#154] +(110) HashAggregate [codegen id : 116] +Input [5]: [channel#133, i_brand_id#134, sum#147, isEmpty#148, sum#149] +Keys [2]: [channel#133, i_brand_id#134] +Functions [2]: [sum(sum_sales#142), sum(number_sales#143)] +Aggregate Attributes [2]: [sum(sum_sales#142)#150, sum(number_sales#143)#151] +Results [6]: [channel#133, i_brand_id#134, null AS i_class_id#152, null AS i_category_id#153, sum(sum_sales#142)#150 AS sum(sum_sales)#154, sum(number_sales#143)#151 AS sum(number_sales)#155] (111) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#155, i_brand_id#156, i_class_id#157, i_category_id#158, sum#159, isEmpty#160, sum#161] - -(112) HashAggregate [codegen id : 322] -Input [7]: [channel#155, i_brand_id#156, i_class_id#157, i_category_id#158, sum#159, isEmpty#160, sum#161] -Keys [4]: [channel#155, i_brand_id#156, i_class_id#157, i_category_id#158] -Functions [2]: [sum(sales#162), sum(number_sales#163)] -Aggregate Attributes [2]: [sum(sales#162)#106, sum(number_sales#163)#107] -Results [3]: [channel#155, sum(sales#162)#106 AS sum_sales#164, sum(number_sales#163)#107 AS number_sales#165] - -(113) HashAggregate [codegen id : 322] -Input [3]: [channel#155, sum_sales#164, number_sales#165] -Keys [1]: [channel#155] -Functions [2]: [partial_sum(sum_sales#164), partial_sum(number_sales#165)] -Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] -Results [4]: [channel#155, sum#169, isEmpty#170, sum#171] +Output [7]: [channel#156, i_brand_id#157, i_class_id#158, i_category_id#159, sum#160, isEmpty#161, sum#162] + +(112) HashAggregate [codegen id : 154] +Input [7]: [channel#156, i_brand_id#157, i_class_id#158, i_category_id#159, sum#160, isEmpty#161, sum#162] +Keys [4]: [channel#156, i_brand_id#157, i_class_id#158, i_category_id#159] +Functions [2]: [sum(sales#163), sum(number_sales#164)] +Aggregate Attributes [2]: [sum(sales#163)#107, sum(number_sales#164)#108] +Results [3]: [channel#156, sum(sales#163)#107 AS sum_sales#165, sum(number_sales#164)#108 AS number_sales#166] + +(113) HashAggregate [codegen id : 154] +Input [3]: [channel#156, sum_sales#165, number_sales#166] +Keys [1]: [channel#156] +Functions [2]: [partial_sum(sum_sales#165), partial_sum(number_sales#166)] +Aggregate Attributes [3]: [sum#167, isEmpty#168, sum#169] +Results [4]: [channel#156, sum#170, isEmpty#171, sum#172] (114) Exchange -Input [4]: [channel#155, sum#169, isEmpty#170, sum#171] -Arguments: hashpartitioning(channel#155, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Input [4]: [channel#156, sum#170, isEmpty#171, sum#172] +Arguments: hashpartitioning(channel#156, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(115) HashAggregate [codegen id : 323] -Input [4]: [channel#155, sum#169, isEmpty#170, sum#171] -Keys [1]: [channel#155] -Functions [2]: [sum(sum_sales#164), sum(number_sales#165)] -Aggregate Attributes [2]: [sum(sum_sales#164)#172, sum(number_sales#165)#173] -Results [6]: [channel#155, null AS i_brand_id#174, null AS i_class_id#175, null AS i_category_id#176, sum(sum_sales#164)#172 AS sum(sum_sales)#177, sum(number_sales#165)#173 AS sum(number_sales)#178] +(115) HashAggregate [codegen id : 155] +Input [4]: [channel#156, sum#170, isEmpty#171, sum#172] +Keys [1]: [channel#156] +Functions [2]: [sum(sum_sales#165), sum(number_sales#166)] +Aggregate Attributes [2]: [sum(sum_sales#165)#173, sum(number_sales#166)#174] +Results [6]: [channel#156, null AS i_brand_id#175, null AS i_class_id#176, null AS i_category_id#177, sum(sum_sales#165)#173 AS sum(sum_sales)#178, sum(number_sales#166)#174 AS sum(number_sales)#179] (116) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#179, i_brand_id#180, i_class_id#181, i_category_id#182, sum#183, isEmpty#184, sum#185] +Output [7]: [channel#180, i_brand_id#181, i_class_id#182, i_category_id#183, sum#184, isEmpty#185, sum#186] -(117) HashAggregate [codegen id : 403] -Input [7]: [channel#179, i_brand_id#180, i_class_id#181, i_category_id#182, sum#183, isEmpty#184, sum#185] -Keys [4]: [channel#179, i_brand_id#180, i_class_id#181, i_category_id#182] -Functions [2]: [sum(sales#186), sum(number_sales#187)] -Aggregate Attributes [2]: [sum(sales#186)#106, sum(number_sales#187)#107] -Results [2]: [sum(sales#186)#106 AS sum_sales#188, sum(number_sales#187)#107 AS number_sales#189] +(117) HashAggregate [codegen id : 193] +Input [7]: [channel#180, i_brand_id#181, i_class_id#182, i_category_id#183, sum#184, isEmpty#185, sum#186] +Keys [4]: [channel#180, i_brand_id#181, i_class_id#182, i_category_id#183] +Functions [2]: [sum(sales#187), sum(number_sales#188)] +Aggregate Attributes [2]: [sum(sales#187)#107, sum(number_sales#188)#108] +Results [2]: [sum(sales#187)#107 AS sum_sales#189, sum(number_sales#188)#108 AS number_sales#190] -(118) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#188, number_sales#189] +(118) HashAggregate [codegen id : 193] +Input [2]: [sum_sales#189, number_sales#190] Keys: [] -Functions [2]: [partial_sum(sum_sales#188), partial_sum(number_sales#189)] -Aggregate Attributes [3]: [sum#190, isEmpty#191, sum#192] -Results [3]: [sum#193, isEmpty#194, sum#195] +Functions [2]: [partial_sum(sum_sales#189), partial_sum(number_sales#190)] +Aggregate Attributes [3]: [sum#191, isEmpty#192, sum#193] +Results [3]: [sum#194, isEmpty#195, sum#196] (119) Exchange -Input [3]: [sum#193, isEmpty#194, sum#195] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] +Input [3]: [sum#194, isEmpty#195, sum#196] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(120) HashAggregate [codegen id : 404] -Input [3]: [sum#193, isEmpty#194, sum#195] +(120) HashAggregate [codegen id : 194] +Input [3]: [sum#194, isEmpty#195, sum#196] Keys: [] -Functions [2]: [sum(sum_sales#188), sum(number_sales#189)] -Aggregate Attributes [2]: [sum(sum_sales#188)#196, sum(number_sales#189)#197] -Results [6]: [null AS channel#198, null AS i_brand_id#199, null AS i_class_id#200, null AS i_category_id#201, sum(sum_sales#188)#196 AS sum(sum_sales)#202, sum(number_sales#189)#197 AS sum(number_sales)#203] +Functions [2]: [sum(sum_sales#189), sum(number_sales#190)] +Aggregate Attributes [2]: [sum(sum_sales#189)#197, sum(number_sales#190)#198] +Results [6]: [null AS channel#199, null AS i_brand_id#200, null AS i_class_id#201, null AS i_category_id#202, sum(sum_sales#189)#197 AS sum(sum_sales)#203, sum(number_sales#190)#198 AS sum(number_sales)#204] (121) Union -(122) HashAggregate [codegen id : 405] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +(122) HashAggregate [codegen id : 195] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#109, number_sales#110] +Keys [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#109, number_sales#110] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#109, number_sales#110] (123) Exchange -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109, 5), ENSURE_REQUIREMENTS, [plan_id=17] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#109, number_sales#110] +Arguments: hashpartitioning(channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#109, number_sales#110, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(124) HashAggregate [codegen id : 406] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +(124) HashAggregate [codegen id : 196] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#109, number_sales#110] +Keys [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#109, number_sales#110] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#109, number_sales#110] (125) TakeOrderedAndProject -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] -Arguments: 100, [channel#51 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#109, number_sales#110] +Arguments: 100, [channel#52 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#109, number_sales#110] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (144) -+- Exchange (143) - +- * HashAggregate (142) - +- Union (141) - :- * Project (130) - : +- * BroadcastHashJoin Inner BuildRight (129) - : :- * ColumnarToRow (127) - : : +- CometScan parquet spark_catalog.default.store_sales (126) - : +- ReusedExchange (128) - :- * Project (135) - : +- * BroadcastHashJoin Inner BuildRight (134) - : :- * ColumnarToRow (132) - : : +- CometScan parquet spark_catalog.default.catalog_sales (131) - : +- ReusedExchange (133) - +- * Project (140) - +- * BroadcastHashJoin Inner BuildRight (139) - :- * ColumnarToRow (137) - : +- CometScan parquet spark_catalog.default.web_sales (136) - +- ReusedExchange (138) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#55, [id=#56] +* HashAggregate (145) ++- Exchange (144) + +- * HashAggregate (143) + +- * ColumnarToRow (142) + +- CometUnion (141) + :- CometProject (129) + : +- CometBroadcastHashJoin (128) + : :- CometScan parquet spark_catalog.default.store_sales (126) + : +- ReusedExchange (127) + :- CometProject (136) + : +- CometBroadcastHashJoin (135) + : :- CometScan parquet spark_catalog.default.catalog_sales (130) + : +- CometBroadcastExchange (134) + : +- CometProject (133) + : +- CometFilter (132) + : +- CometScan parquet spark_catalog.default.date_dim (131) + +- CometProject (140) + +- CometBroadcastHashJoin (139) + :- CometScan parquet spark_catalog.default.web_sales (137) + +- ReusedExchange (138) (126) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#204, ss_list_price#205, ss_sold_date_sk#206] +Output [3]: [ss_quantity#205, ss_list_price#206, ss_sold_date_sk#207] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#206), dynamicpruningexpression(ss_sold_date_sk#206 IN dynamicpruning#207)] +PartitionFilters: [isnotnull(ss_sold_date_sk#207), dynamicpruningexpression(ss_sold_date_sk#207 IN dynamicpruning#208)] ReadSchema: struct -(127) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#204, ss_list_price#205, ss_sold_date_sk#206] - -(128) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#208] +(127) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#209] -(129) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#206] -Right keys [1]: [d_date_sk#208] -Join type: Inner -Join condition: None +(128) CometBroadcastHashJoin +Left output [3]: [ss_quantity#205, ss_list_price#206, ss_sold_date_sk#207] +Right output [1]: [d_date_sk#209] +Arguments: [ss_sold_date_sk#207], [d_date_sk#209], Inner, BuildRight -(130) Project [codegen id : 2] -Output [2]: [ss_quantity#204 AS quantity#209, ss_list_price#205 AS list_price#210] -Input [4]: [ss_quantity#204, ss_list_price#205, ss_sold_date_sk#206, d_date_sk#208] +(129) CometProject +Input [4]: [ss_quantity#205, ss_list_price#206, ss_sold_date_sk#207, d_date_sk#209] +Arguments: [quantity#210, list_price#211], [ss_quantity#205 AS quantity#210, ss_list_price#206 AS list_price#211] -(131) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#211, cs_list_price#212, cs_sold_date_sk#213] +(130) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#212, cs_list_price#213, cs_sold_date_sk#214] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#213), dynamicpruningexpression(cs_sold_date_sk#213 IN dynamicpruning#214)] +PartitionFilters: [isnotnull(cs_sold_date_sk#214), dynamicpruningexpression(cs_sold_date_sk#214 IN dynamicpruning#215)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#211, cs_list_price#212, cs_sold_date_sk#213] +(131) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#216, d_year#217] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct + +(132) CometFilter +Input [2]: [d_date_sk#216, d_year#217] +Condition : (((isnotnull(d_year#217) AND (d_year#217 >= 1998)) AND (d_year#217 <= 2000)) AND isnotnull(d_date_sk#216)) -(133) ReusedExchange [Reuses operator id: 149] -Output [1]: [d_date_sk#215] +(133) CometProject +Input [2]: [d_date_sk#216, d_year#217] +Arguments: [d_date_sk#216], [d_date_sk#216] -(134) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#213] -Right keys [1]: [d_date_sk#215] -Join type: Inner -Join condition: None +(134) CometBroadcastExchange +Input [1]: [d_date_sk#216] +Arguments: [d_date_sk#216] -(135) Project [codegen id : 4] -Output [2]: [cs_quantity#211 AS quantity#216, cs_list_price#212 AS list_price#217] -Input [4]: [cs_quantity#211, cs_list_price#212, cs_sold_date_sk#213, d_date_sk#215] +(135) CometBroadcastHashJoin +Left output [3]: [cs_quantity#212, cs_list_price#213, cs_sold_date_sk#214] +Right output [1]: [d_date_sk#216] +Arguments: [cs_sold_date_sk#214], [d_date_sk#216], Inner, BuildRight -(136) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#218, ws_list_price#219, ws_sold_date_sk#220] +(136) CometProject +Input [4]: [cs_quantity#212, cs_list_price#213, cs_sold_date_sk#214, d_date_sk#216] +Arguments: [quantity#218, list_price#219], [cs_quantity#212 AS quantity#218, cs_list_price#213 AS list_price#219] + +(137) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#220, ws_list_price#221, ws_sold_date_sk#222] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#220), dynamicpruningexpression(ws_sold_date_sk#220 IN dynamicpruning#221)] +PartitionFilters: [isnotnull(ws_sold_date_sk#222), dynamicpruningexpression(ws_sold_date_sk#222 IN dynamicpruning#223)] ReadSchema: struct -(137) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#218, ws_list_price#219, ws_sold_date_sk#220] +(138) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#224] -(138) ReusedExchange [Reuses operator id: 149] -Output [1]: [d_date_sk#222] +(139) CometBroadcastHashJoin +Left output [3]: [ws_quantity#220, ws_list_price#221, ws_sold_date_sk#222] +Right output [1]: [d_date_sk#224] +Arguments: [ws_sold_date_sk#222], [d_date_sk#224], Inner, BuildRight -(139) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#220] -Right keys [1]: [d_date_sk#222] -Join type: Inner -Join condition: None +(140) CometProject +Input [4]: [ws_quantity#220, ws_list_price#221, ws_sold_date_sk#222, d_date_sk#224] +Arguments: [quantity#225, list_price#226], [ws_quantity#220 AS quantity#225, ws_list_price#221 AS list_price#226] -(140) Project [codegen id : 6] -Output [2]: [ws_quantity#218 AS quantity#223, ws_list_price#219 AS list_price#224] -Input [4]: [ws_quantity#218, ws_list_price#219, ws_sold_date_sk#220, d_date_sk#222] +(141) CometUnion +Child 0 Input [2]: [quantity#210, list_price#211] +Child 1 Input [2]: [quantity#218, list_price#219] +Child 2 Input [2]: [quantity#225, list_price#226] -(141) Union +(142) ColumnarToRow [codegen id : 1] +Input [2]: [quantity#210, list_price#211] -(142) HashAggregate [codegen id : 7] -Input [2]: [quantity#209, list_price#210] +(143) HashAggregate [codegen id : 1] +Input [2]: [quantity#210, list_price#211] Keys: [] -Functions [1]: [partial_avg((cast(quantity#209 as decimal(10,0)) * list_price#210))] -Aggregate Attributes [2]: [sum#225, count#226] -Results [2]: [sum#227, count#228] +Functions [1]: [partial_avg((cast(quantity#210 as decimal(10,0)) * list_price#211))] +Aggregate Attributes [2]: [sum#227, count#228] +Results [2]: [sum#229, count#230] -(143) Exchange -Input [2]: [sum#227, count#228] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] +(144) Exchange +Input [2]: [sum#229, count#230] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] -(144) HashAggregate [codegen id : 8] -Input [2]: [sum#227, count#228] +(145) HashAggregate [codegen id : 2] +Input [2]: [sum#229, count#230] Keys: [] -Functions [1]: [avg((cast(quantity#209 as decimal(10,0)) * list_price#210))] -Aggregate Attributes [1]: [avg((cast(quantity#209 as decimal(10,0)) * list_price#210))#229] -Results [1]: [avg((cast(quantity#209 as decimal(10,0)) * list_price#210))#229 AS average_sales#230] +Functions [1]: [avg((cast(quantity#210 as decimal(10,0)) * list_price#211))] +Aggregate Attributes [1]: [avg((cast(quantity#210 as decimal(10,0)) * list_price#211))#231] +Results [1]: [avg((cast(quantity#210 as decimal(10,0)) * list_price#211))#231 AS average_sales#232] -Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#206 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#207 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 131 Hosting Expression = cs_sold_date_sk#213 IN dynamicpruning#214 -BroadcastExchange (149) -+- * ColumnarToRow (148) - +- CometProject (147) - +- CometFilter (146) - +- CometScan parquet spark_catalog.default.date_dim (145) +Subquery:3 Hosting operator id = 130 Hosting Expression = cs_sold_date_sk#214 IN dynamicpruning#215 +BroadcastExchange (150) ++- * ColumnarToRow (149) + +- CometProject (148) + +- CometFilter (147) + +- CometScan parquet spark_catalog.default.date_dim (146) -(145) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#215, d_year#231] +(146) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#216, d_year#217] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(146) CometFilter -Input [2]: [d_date_sk#215, d_year#231] -Condition : (((isnotnull(d_year#231) AND (d_year#231 >= 1998)) AND (d_year#231 <= 2000)) AND isnotnull(d_date_sk#215)) +(147) CometFilter +Input [2]: [d_date_sk#216, d_year#217] +Condition : (((isnotnull(d_year#217) AND (d_year#217 >= 1998)) AND (d_year#217 <= 2000)) AND isnotnull(d_date_sk#216)) -(147) CometProject -Input [2]: [d_date_sk#215, d_year#231] -Arguments: [d_date_sk#215], [d_date_sk#215] +(148) CometProject +Input [2]: [d_date_sk#216, d_year#217] +Arguments: [d_date_sk#216], [d_date_sk#216] -(148) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#215] +(149) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#216] -(149) BroadcastExchange -Input [1]: [d_date_sk#215] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] +(150) BroadcastExchange +Input [1]: [d_date_sk#216] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:4 Hosting operator id = 136 Hosting Expression = ws_sold_date_sk#220 IN dynamicpruning#214 +Subquery:4 Hosting operator id = 137 Hosting Expression = ws_sold_date_sk#222 IN dynamicpruning#215 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (154) -+- * ColumnarToRow (153) - +- CometProject (152) - +- CometFilter (151) - +- CometScan parquet spark_catalog.default.date_dim (150) +BroadcastExchange (155) ++- * ColumnarToRow (154) + +- CometProject (153) + +- CometFilter (152) + +- CometScan parquet spark_catalog.default.date_dim (151) -(150) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#42, d_year#232, d_moy#233] +(151) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#233, d_moy#234] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(151) CometFilter -Input [3]: [d_date_sk#42, d_year#232, d_moy#233] -Condition : ((((isnotnull(d_year#232) AND isnotnull(d_moy#233)) AND (d_year#232 = 2000)) AND (d_moy#233 = 11)) AND isnotnull(d_date_sk#42)) +(152) CometFilter +Input [3]: [d_date_sk#43, d_year#233, d_moy#234] +Condition : ((((isnotnull(d_year#233) AND isnotnull(d_moy#234)) AND (d_year#233 = 2000)) AND (d_moy#234 = 11)) AND isnotnull(d_date_sk#43)) -(152) CometProject -Input [3]: [d_date_sk#42, d_year#232, d_moy#233] -Arguments: [d_date_sk#42], [d_date_sk#42] +(153) CometProject +Input [3]: [d_date_sk#43, d_year#233, d_moy#234] +Arguments: [d_date_sk#43], [d_date_sk#43] -(153) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(154) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(154) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] +(155) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (159) -+- * ColumnarToRow (158) - +- CometProject (157) - +- CometFilter (156) - +- CometScan parquet spark_catalog.default.date_dim (155) +BroadcastExchange (160) ++- * ColumnarToRow (159) + +- CometProject (158) + +- CometFilter (157) + +- CometScan parquet spark_catalog.default.date_dim (156) -(155) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#234] +(156) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#235] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(156) CometFilter -Input [2]: [d_date_sk#25, d_year#234] -Condition : (((isnotnull(d_year#234) AND (d_year#234 >= 1999)) AND (d_year#234 <= 2001)) AND isnotnull(d_date_sk#25)) +(157) CometFilter +Input [2]: [d_date_sk#26, d_year#235] +Condition : (((isnotnull(d_year#235) AND (d_year#235 >= 1999)) AND (d_year#235 <= 2001)) AND isnotnull(d_date_sk#26)) -(157) CometProject -Input [2]: [d_date_sk#25, d_year#234] -Arguments: [d_date_sk#25], [d_date_sk#25] +(158) CometProject +Input [2]: [d_date_sk#26, d_year#235] +Arguments: [d_date_sk#26], [d_date_sk#26] -(158) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(159) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(159) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] +(160) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=18] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt index a203f9620b..6e0ac9858f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt @@ -1,69 +1,61 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (406) + WholeStageCodegen (196) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) + WholeStageCodegen (195) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (80) + WholeStageCodegen (38) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) + WholeStageCodegen (37) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (26) + WholeStageCodegen (12) Filter [sales] Subquery #3 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #14 - WholeStageCodegen (7) + Exchange #15 + WholeStageCodegen (1) HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #15 + ColumnarToRow + InputAdapter + CometUnion + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #12 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #16 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #17 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + ReusedExchange [d_date_sk] #17 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) + WholeStageCodegen (11) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -84,7 +76,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 - WholeStageCodegen (11) + WholeStageCodegen (4) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow @@ -93,19 +85,19 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 - WholeStageCodegen (10) + WholeStageCodegen (3) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 @@ -116,56 +108,43 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 + CometBroadcastExchange #9 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #10 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange #11 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #12 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #12 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #13 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [d_date_sk] #12 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) + BroadcastExchange #14 + WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter @@ -175,13 +154,13 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [ss_item_sk] #5 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) + WholeStageCodegen (24) Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) + Exchange [i_brand_id,i_class_id,i_category_id] #18 + WholeStageCodegen (23) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -196,16 +175,16 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) + WholeStageCodegen (36) Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) + Exchange [i_brand_id,i_class_id,i_category_id] #19 + WholeStageCodegen (35) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -220,41 +199,41 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) + WholeStageCodegen (77) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) + Exchange [channel,i_brand_id,i_class_id] #20 + WholeStageCodegen (76) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) + WholeStageCodegen (116) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id] #19 - WholeStageCodegen (241) + Exchange [channel,i_brand_id] #21 + WholeStageCodegen (115) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) + WholeStageCodegen (155) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel] #20 - WholeStageCodegen (322) + Exchange [channel] #22 + WholeStageCodegen (154) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) + WholeStageCodegen (194) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange #21 - WholeStageCodegen (403) + Exchange #23 + WholeStageCodegen (193) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt index 8bcc0c6bac..36ccc06dea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt @@ -1,157 +1,151 @@ == Physical Plan == -TakeOrderedAndProject (153) -+- Union (152) - :- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * ColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan parquet spark_catalog.default.customer (11) - : : : : +- BroadcastExchange (21) - : : : : +- * ColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) - : : : +- BroadcastExchange (27) - : : : +- * ColumnarToRow (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.customer_address (24) - : : +- ReusedExchange (30) - : +- BroadcastExchange (36) - : +- * ColumnarToRow (35) - : +- CometFilter (34) - : +- CometScan parquet spark_catalog.default.item (33) - :- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * Project (53) - : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : :- * Project (50) - : : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : : :- * Project (47) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : : : :- * ColumnarToRow (44) - : : : : : : : +- CometFilter (43) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (42) - : : : : : : +- ReusedExchange (45) - : : : : : +- ReusedExchange (48) - : : : : +- ReusedExchange (51) - : : : +- BroadcastExchange (57) - : : : +- * ColumnarToRow (56) - : : : +- CometFilter (55) - : : : +- CometScan parquet spark_catalog.default.customer_address (54) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - :- * HashAggregate (96) - : +- Exchange (95) - : +- * HashAggregate (94) - : +- * Project (93) - : +- * BroadcastHashJoin Inner BuildRight (92) - : :- * Project (90) - : : +- * BroadcastHashJoin Inner BuildRight (89) - : : :- * Project (87) - : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : :- * Project (80) - : : : : +- * BroadcastHashJoin Inner BuildRight (79) - : : : : :- * Project (77) - : : : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : : : :- * Project (74) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : : : : :- * ColumnarToRow (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (69) - : : : : : : +- ReusedExchange (72) - : : : : : +- ReusedExchange (75) - : : : : +- ReusedExchange (78) - : : : +- BroadcastExchange (85) - : : : +- * ColumnarToRow (84) - : : : +- CometProject (83) - : : : +- CometFilter (82) - : : : +- CometScan parquet spark_catalog.default.customer_address (81) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- * HashAggregate (124) - : +- Exchange (123) - : +- * HashAggregate (122) - : +- * Project (121) - : +- * BroadcastHashJoin Inner BuildRight (120) - : :- * Project (118) - : : +- * BroadcastHashJoin Inner BuildRight (117) - : : :- * Project (115) - : : : +- * BroadcastHashJoin Inner BuildRight (114) - : : : :- * Project (108) - : : : : +- * BroadcastHashJoin Inner BuildRight (107) - : : : : :- * Project (105) - : : : : : +- * BroadcastHashJoin Inner BuildRight (104) - : : : : : :- * Project (102) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (101) - : : : : : : :- * ColumnarToRow (99) - : : : : : : : +- CometFilter (98) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (97) - : : : : : : +- ReusedExchange (100) - : : : : : +- ReusedExchange (103) - : : : : +- ReusedExchange (106) - : : : +- BroadcastExchange (113) - : : : +- * ColumnarToRow (112) - : : : +- CometProject (111) - : : : +- CometFilter (110) - : : : +- CometScan parquet spark_catalog.default.customer_address (109) - : : +- ReusedExchange (116) - : +- ReusedExchange (119) - +- * HashAggregate (151) - +- Exchange (150) - +- * HashAggregate (149) - +- * Project (148) - +- * BroadcastHashJoin Inner BuildRight (147) - :- * Project (142) - : +- * BroadcastHashJoin Inner BuildRight (141) - : :- * Project (139) - : : +- * BroadcastHashJoin Inner BuildRight (138) - : : :- * Project (136) - : : : +- * BroadcastHashJoin Inner BuildRight (135) - : : : :- * Project (133) - : : : : +- * BroadcastHashJoin Inner BuildRight (132) - : : : : :- * Project (130) - : : : : : +- * BroadcastHashJoin Inner BuildRight (129) - : : : : : :- * ColumnarToRow (127) - : : : : : : +- CometFilter (126) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (125) - : : : : : +- ReusedExchange (128) - : : : : +- ReusedExchange (131) - : : : +- ReusedExchange (134) - : : +- ReusedExchange (137) - : +- ReusedExchange (140) - +- BroadcastExchange (146) - +- * ColumnarToRow (145) - +- CometFilter (144) - +- CometScan parquet spark_catalog.default.item (143) +TakeOrderedAndProject (147) ++- Union (146) + :- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * ColumnarToRow (36) + : +- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometProject (19) + : : : : +- CometBroadcastHashJoin (18) + : : : : :- CometProject (14) + : : : : : +- CometBroadcastHashJoin (13) + : : : : : :- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : +- CometBroadcastExchange (6) + : : : : : : +- CometProject (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : : : : +- CometBroadcastExchange (12) + : : : : : +- CometProject (11) + : : : : : +- CometFilter (10) + : : : : : +- CometScan parquet spark_catalog.default.customer (9) + : : : : +- CometBroadcastExchange (17) + : : : : +- CometFilter (16) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (15) + : : : +- CometBroadcastExchange (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.customer_address (20) + : : +- CometBroadcastExchange (28) + : : +- CometProject (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- CometBroadcastExchange (33) + : +- CometFilter (32) + : +- CometScan parquet spark_catalog.default.item (31) + :- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * ColumnarToRow (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (58) + : : +- CometBroadcastHashJoin (57) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometProject (50) + : : : : +- CometBroadcastHashJoin (49) + : : : : :- CometProject (47) + : : : : : +- CometBroadcastHashJoin (46) + : : : : : :- CometProject (44) + : : : : : : +- CometBroadcastHashJoin (43) + : : : : : : :- CometFilter (41) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) + : : : : : : +- ReusedExchange (42) + : : : : : +- ReusedExchange (45) + : : : : +- ReusedExchange (48) + : : : +- CometBroadcastExchange (53) + : : : +- CometFilter (52) + : : : +- CometScan parquet spark_catalog.default.customer_address (51) + : : +- ReusedExchange (56) + : +- ReusedExchange (59) + :- * HashAggregate (92) + : +- Exchange (91) + : +- * HashAggregate (90) + : +- * ColumnarToRow (89) + : +- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometProject (85) + : : +- CometBroadcastHashJoin (84) + : : :- CometProject (82) + : : : +- CometBroadcastHashJoin (81) + : : : :- CometProject (76) + : : : : +- CometBroadcastHashJoin (75) + : : : : :- CometProject (73) + : : : : : +- CometBroadcastHashJoin (72) + : : : : : :- CometProject (70) + : : : : : : +- CometBroadcastHashJoin (69) + : : : : : : :- CometFilter (67) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (66) + : : : : : : +- ReusedExchange (68) + : : : : : +- ReusedExchange (71) + : : : : +- ReusedExchange (74) + : : : +- CometBroadcastExchange (80) + : : : +- CometProject (79) + : : : +- CometFilter (78) + : : : +- CometScan parquet spark_catalog.default.customer_address (77) + : : +- ReusedExchange (83) + : +- ReusedExchange (86) + :- * HashAggregate (119) + : +- Exchange (118) + : +- * HashAggregate (117) + : +- * ColumnarToRow (116) + : +- CometProject (115) + : +- CometBroadcastHashJoin (114) + : :- CometProject (112) + : : +- CometBroadcastHashJoin (111) + : : :- CometProject (109) + : : : +- CometBroadcastHashJoin (108) + : : : :- CometProject (103) + : : : : +- CometBroadcastHashJoin (102) + : : : : :- CometProject (100) + : : : : : +- CometBroadcastHashJoin (99) + : : : : : :- CometProject (97) + : : : : : : +- CometBroadcastHashJoin (96) + : : : : : : :- CometFilter (94) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (93) + : : : : : : +- ReusedExchange (95) + : : : : : +- ReusedExchange (98) + : : : : +- ReusedExchange (101) + : : : +- CometBroadcastExchange (107) + : : : +- CometProject (106) + : : : +- CometFilter (105) + : : : +- CometScan parquet spark_catalog.default.customer_address (104) + : : +- ReusedExchange (110) + : +- ReusedExchange (113) + +- * HashAggregate (145) + +- Exchange (144) + +- * HashAggregate (143) + +- * ColumnarToRow (142) + +- CometProject (141) + +- CometBroadcastHashJoin (140) + :- CometProject (136) + : +- CometBroadcastHashJoin (135) + : :- CometProject (133) + : : +- CometBroadcastHashJoin (132) + : : :- CometProject (130) + : : : +- CometBroadcastHashJoin (129) + : : : :- CometProject (127) + : : : : +- CometBroadcastHashJoin (126) + : : : : :- CometProject (124) + : : : : : +- CometBroadcastHashJoin (123) + : : : : : :- CometFilter (121) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (120) + : : : : : +- ReusedExchange (122) + : : : : +- ReusedExchange (125) + : : : +- ReusedExchange (128) + : : +- ReusedExchange (131) + : +- ReusedExchange (134) + +- CometBroadcastExchange (139) + +- CometFilter (138) + +- CometScan parquet spark_catalog.default.item (137) (1) Scan parquet spark_catalog.default.catalog_sales @@ -166,744 +160,703 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = M)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#11)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) ColumnarToRow [codegen id : 1] -Input [2]: [cd_demo_sk#11, cd_dep_count#14] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#11, cd_dep_count#14] -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Right output [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +(8) CometProject Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(11) Scan parquet spark_catalog.default.customer +(9) Scan parquet spark_catalog.default.customer Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) -(13) CometProject +(11) CometProject Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) ColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) BroadcastExchange +(12) CometBroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +(14) CometProject Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(18) Scan parquet spark_catalog.default.customer_demographics +(15) Scan parquet spark_catalog.default.customer_demographics Output [1]: [cd_demo_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(19) CometFilter +(16) CometFilter Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) ColumnarToRow [codegen id : 3] +(17) CometBroadcastExchange Input [1]: [cd_demo_sk#20] +Arguments: [cd_demo_sk#20] -(21) BroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [cd_demo_sk#20] +Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +(19) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(24) Scan parquet spark_catalog.default.customer_address +(20) Scan parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(25) CometFilter +(21) CometFilter Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(26) ColumnarToRow [codegen id : 4] -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] - -(27) BroadcastExchange +(22) CometBroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -(28) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None +(23) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight -(29) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +(24) CometProject Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] -(30) ReusedExchange [Reuses operator id: 158] -Output [1]: [d_date_sk#25] +(25) Scan parquet spark_catalog.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,2001), IsNotNull(d_date_sk)] +ReadSchema: struct + +(26) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#25)) -(31) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(27) CometProject +Input [2]: [d_date_sk#25, d_year#26] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(28) CometBroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: [d_date_sk#25] -(32) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +(29) CometBroadcastHashJoin +Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#9], [d_date_sk#25], Inner, BuildRight + +(30) CometProject Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, d_date_sk#25] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] -(33) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#26, i_item_id#27] +(31) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#27, i_item_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) CometFilter -Input [2]: [i_item_sk#26, i_item_id#27] -Condition : isnotnull(i_item_sk#26) - -(35) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#26, i_item_id#27] - -(36) BroadcastExchange -Input [2]: [i_item_sk#26, i_item_id#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 7] -Output [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#26, i_item_id#27] - -(39) HashAggregate [codegen id : 7] -Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] -Keys [4]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22] -Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Results [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] - -(40) Exchange -Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, ca_county#22, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(41) HashAggregate [codegen id : 8] -Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Keys [4]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22] -Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] -Aggregate Attributes [7]: [avg(agg1#28)#63, avg(agg2#29)#64, avg(agg3#30)#65, avg(agg4#31)#66, avg(agg5#32)#67, avg(agg6#33)#68, avg(agg7#34)#69] -Results [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, avg(agg1#28)#63 AS agg1#70, avg(agg2#29)#64 AS agg2#71, avg(agg3#30)#65 AS agg3#72, avg(agg4#31)#66 AS agg4#73, avg(agg5#32)#67 AS agg5#74, avg(agg6#33)#68 AS agg6#75, avg(agg7#34)#69 AS agg7#76] - -(42) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#77, cs_bill_cdemo_sk#78, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85] +(32) CometFilter +Input [2]: [i_item_sk#27, i_item_id#28] +Condition : isnotnull(i_item_sk#27) + +(33) CometBroadcastExchange +Input [2]: [i_item_sk#27, i_item_id#28] +Arguments: [i_item_sk#27, i_item_id#28] + +(34) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Right output [2]: [i_item_sk#27, i_item_id#28] +Arguments: [cs_item_sk#3], [i_item_sk#27], Inner, BuildRight + +(35) CometProject +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#27, i_item_id#28] +Arguments: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35], [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, cast(cs_quantity#4 as decimal(12,2)) AS agg1#29, cast(cs_list_price#5 as decimal(12,2)) AS agg2#30, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#31, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#32, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#33, cast(c_birth_year#19 as decimal(12,2)) AS agg6#34, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#35] + +(36) ColumnarToRow [codegen id : 1] +Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35] + +(37) HashAggregate [codegen id : 1] +Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35] +Keys [4]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22] +Functions [7]: [partial_avg(agg1#29), partial_avg(agg2#30), partial_avg(agg3#31), partial_avg(agg4#32), partial_avg(agg5#33), partial_avg(agg6#34), partial_avg(agg7#35)] +Aggregate Attributes [14]: [sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] +Results [18]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63] + +(38) Exchange +Input [18]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63] +Arguments: hashpartitioning(i_item_id#28, ca_country#24, ca_state#23, ca_county#22, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(39) HashAggregate [codegen id : 2] +Input [18]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63] +Keys [4]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22] +Functions [7]: [avg(agg1#29), avg(agg2#30), avg(agg3#31), avg(agg4#32), avg(agg5#33), avg(agg6#34), avg(agg7#35)] +Aggregate Attributes [7]: [avg(agg1#29)#64, avg(agg2#30)#65, avg(agg3#31)#66, avg(agg4#32)#67, avg(agg5#33)#68, avg(agg6#34)#69, avg(agg7#35)#70] +Results [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, avg(agg1#29)#64 AS agg1#71, avg(agg2#30)#65 AS agg2#72, avg(agg3#31)#66 AS agg3#73, avg(agg4#32)#67 AS agg4#74, avg(agg5#33)#68 AS agg5#75, avg(agg6#34)#69 AS agg6#76, avg(agg7#35)#70 AS agg7#77] + +(40) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#78, cs_bill_cdemo_sk#79, cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#85), dynamicpruningexpression(cs_sold_date_sk#85 IN dynamicpruning#86)] +PartitionFilters: [isnotnull(cs_sold_date_sk#86), dynamicpruningexpression(cs_sold_date_sk#86 IN dynamicpruning#87)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(43) CometFilter -Input [9]: [cs_bill_customer_sk#77, cs_bill_cdemo_sk#78, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85] -Condition : ((isnotnull(cs_bill_cdemo_sk#78) AND isnotnull(cs_bill_customer_sk#77)) AND isnotnull(cs_item_sk#79)) - -(44) ColumnarToRow [codegen id : 15] -Input [9]: [cs_bill_customer_sk#77, cs_bill_cdemo_sk#78, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85] +(41) CometFilter +Input [9]: [cs_bill_customer_sk#78, cs_bill_cdemo_sk#79, cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86] +Condition : ((isnotnull(cs_bill_cdemo_sk#79) AND isnotnull(cs_bill_customer_sk#78)) AND isnotnull(cs_item_sk#80)) -(45) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#87, cd_dep_count#88] +(42) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#88, cd_dep_count#89] -(46) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#78] -Right keys [1]: [cd_demo_sk#87] -Join type: Inner -Join condition: None +(43) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#78, cs_bill_cdemo_sk#79, cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86] +Right output [2]: [cd_demo_sk#88, cd_dep_count#89] +Arguments: [cs_bill_cdemo_sk#79], [cd_demo_sk#88], Inner, BuildRight -(47) Project [codegen id : 15] -Output [9]: [cs_bill_customer_sk#77, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88] -Input [11]: [cs_bill_customer_sk#77, cs_bill_cdemo_sk#78, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_demo_sk#87, cd_dep_count#88] +(44) CometProject +Input [11]: [cs_bill_customer_sk#78, cs_bill_cdemo_sk#79, cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_demo_sk#88, cd_dep_count#89] +Arguments: [cs_bill_customer_sk#78, cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89], [cs_bill_customer_sk#78, cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89] -(48) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#89, c_current_cdemo_sk#90, c_current_addr_sk#91, c_birth_year#92] +(45) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#90, c_current_cdemo_sk#91, c_current_addr_sk#92, c_birth_year#93] -(49) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#77] -Right keys [1]: [c_customer_sk#89] -Join type: Inner -Join condition: None +(46) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#78, cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89] +Right output [4]: [c_customer_sk#90, c_current_cdemo_sk#91, c_current_addr_sk#92, c_birth_year#93] +Arguments: [cs_bill_customer_sk#78], [c_customer_sk#90], Inner, BuildRight -(50) Project [codegen id : 15] -Output [11]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_current_cdemo_sk#90, c_current_addr_sk#91, c_birth_year#92] -Input [13]: [cs_bill_customer_sk#77, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_customer_sk#89, c_current_cdemo_sk#90, c_current_addr_sk#91, c_birth_year#92] +(47) CometProject +Input [13]: [cs_bill_customer_sk#78, cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_customer_sk#90, c_current_cdemo_sk#91, c_current_addr_sk#92, c_birth_year#93] +Arguments: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_current_cdemo_sk#91, c_current_addr_sk#92, c_birth_year#93], [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_current_cdemo_sk#91, c_current_addr_sk#92, c_birth_year#93] -(51) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#93] +(48) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#94] -(52) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#90] -Right keys [1]: [cd_demo_sk#93] -Join type: Inner -Join condition: None +(49) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_current_cdemo_sk#91, c_current_addr_sk#92, c_birth_year#93] +Right output [1]: [cd_demo_sk#94] +Arguments: [c_current_cdemo_sk#91], [cd_demo_sk#94], Inner, BuildRight -(53) Project [codegen id : 15] -Output [10]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_current_addr_sk#91, c_birth_year#92] -Input [12]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_current_cdemo_sk#90, c_current_addr_sk#91, c_birth_year#92, cd_demo_sk#93] +(50) CometProject +Input [12]: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_current_cdemo_sk#91, c_current_addr_sk#92, c_birth_year#93, cd_demo_sk#94] +Arguments: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_current_addr_sk#92, c_birth_year#93], [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_current_addr_sk#92, c_birth_year#93] -(54) Scan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#94, ca_state#95, ca_country#96] +(51) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#95, ca_state#96, ca_country#97] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(55) CometFilter -Input [3]: [ca_address_sk#94, ca_state#95, ca_country#96] -Condition : (ca_state#95 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#94)) - -(56) ColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#94, ca_state#95, ca_country#96] - -(57) BroadcastExchange -Input [3]: [ca_address_sk#94, ca_state#95, ca_country#96] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(58) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#91] -Right keys [1]: [ca_address_sk#94] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 15] -Output [11]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_birth_year#92, ca_state#95, ca_country#96] -Input [13]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_current_addr_sk#91, c_birth_year#92, ca_address_sk#94, ca_state#95, ca_country#96] - -(60) ReusedExchange [Reuses operator id: 158] -Output [1]: [d_date_sk#97] - -(61) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#85] -Right keys [1]: [d_date_sk#97] -Join type: Inner -Join condition: None - -(62) Project [codegen id : 15] -Output [10]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cd_dep_count#88, c_birth_year#92, ca_state#95, ca_country#96] -Input [12]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_birth_year#92, ca_state#95, ca_country#96, d_date_sk#97] - -(63) ReusedExchange [Reuses operator id: 36] -Output [2]: [i_item_sk#98, i_item_id#99] - -(64) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#79] -Right keys [1]: [i_item_sk#98] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 15] -Output [10]: [i_item_id#99, ca_country#96, ca_state#95, cast(cs_quantity#80 as decimal(12,2)) AS agg1#100, cast(cs_list_price#81 as decimal(12,2)) AS agg2#101, cast(cs_coupon_amt#83 as decimal(12,2)) AS agg3#102, cast(cs_sales_price#82 as decimal(12,2)) AS agg4#103, cast(cs_net_profit#84 as decimal(12,2)) AS agg5#104, cast(c_birth_year#92 as decimal(12,2)) AS agg6#105, cast(cd_dep_count#88 as decimal(12,2)) AS agg7#106] -Input [12]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cd_dep_count#88, c_birth_year#92, ca_state#95, ca_country#96, i_item_sk#98, i_item_id#99] - -(66) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#99, ca_country#96, ca_state#95, agg1#100, agg2#101, agg3#102, agg4#103, agg5#104, agg6#105, agg7#106] -Keys [3]: [i_item_id#99, ca_country#96, ca_state#95] -Functions [7]: [partial_avg(agg1#100), partial_avg(agg2#101), partial_avg(agg3#102), partial_avg(agg4#103), partial_avg(agg5#104), partial_avg(agg6#105), partial_avg(agg7#106)] -Aggregate Attributes [14]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] -Results [17]: [i_item_id#99, ca_country#96, ca_state#95, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] - -(67) Exchange -Input [17]: [i_item_id#99, ca_country#96, ca_state#95, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] -Arguments: hashpartitioning(i_item_id#99, ca_country#96, ca_state#95, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(68) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#99, ca_country#96, ca_state#95, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] -Keys [3]: [i_item_id#99, ca_country#96, ca_state#95] -Functions [7]: [avg(agg1#100), avg(agg2#101), avg(agg3#102), avg(agg4#103), avg(agg5#104), avg(agg6#105), avg(agg7#106)] -Aggregate Attributes [7]: [avg(agg1#100)#135, avg(agg2#101)#136, avg(agg3#102)#137, avg(agg4#103)#138, avg(agg5#104)#139, avg(agg6#105)#140, avg(agg7#106)#141] -Results [11]: [i_item_id#99, ca_country#96, ca_state#95, null AS county#142, avg(agg1#100)#135 AS agg1#143, avg(agg2#101)#136 AS agg2#144, avg(agg3#102)#137 AS agg3#145, avg(agg4#103)#138 AS agg4#146, avg(agg5#104)#139 AS agg5#147, avg(agg6#105)#140 AS agg6#148, avg(agg7#106)#141 AS agg7#149] - -(69) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158] +(52) CometFilter +Input [3]: [ca_address_sk#95, ca_state#96, ca_country#97] +Condition : (ca_state#96 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#95)) + +(53) CometBroadcastExchange +Input [3]: [ca_address_sk#95, ca_state#96, ca_country#97] +Arguments: [ca_address_sk#95, ca_state#96, ca_country#97] + +(54) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_current_addr_sk#92, c_birth_year#93] +Right output [3]: [ca_address_sk#95, ca_state#96, ca_country#97] +Arguments: [c_current_addr_sk#92], [ca_address_sk#95], Inner, BuildRight + +(55) CometProject +Input [13]: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_current_addr_sk#92, c_birth_year#93, ca_address_sk#95, ca_state#96, ca_country#97] +Arguments: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_birth_year#93, ca_state#96, ca_country#97], [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_birth_year#93, ca_state#96, ca_country#97] + +(56) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#98] + +(57) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_birth_year#93, ca_state#96, ca_country#97] +Right output [1]: [d_date_sk#98] +Arguments: [cs_sold_date_sk#86], [d_date_sk#98], Inner, BuildRight + +(58) CometProject +Input [12]: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cs_sold_date_sk#86, cd_dep_count#89, c_birth_year#93, ca_state#96, ca_country#97, d_date_sk#98] +Arguments: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cd_dep_count#89, c_birth_year#93, ca_state#96, ca_country#97], [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cd_dep_count#89, c_birth_year#93, ca_state#96, ca_country#97] + +(59) ReusedExchange [Reuses operator id: 33] +Output [2]: [i_item_sk#99, i_item_id#100] + +(60) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cd_dep_count#89, c_birth_year#93, ca_state#96, ca_country#97] +Right output [2]: [i_item_sk#99, i_item_id#100] +Arguments: [cs_item_sk#80], [i_item_sk#99], Inner, BuildRight + +(61) CometProject +Input [12]: [cs_item_sk#80, cs_quantity#81, cs_list_price#82, cs_sales_price#83, cs_coupon_amt#84, cs_net_profit#85, cd_dep_count#89, c_birth_year#93, ca_state#96, ca_country#97, i_item_sk#99, i_item_id#100] +Arguments: [i_item_id#100, ca_country#97, ca_state#96, agg1#101, agg2#102, agg3#103, agg4#104, agg5#105, agg6#106, agg7#107], [i_item_id#100, ca_country#97, ca_state#96, cast(cs_quantity#81 as decimal(12,2)) AS agg1#101, cast(cs_list_price#82 as decimal(12,2)) AS agg2#102, cast(cs_coupon_amt#84 as decimal(12,2)) AS agg3#103, cast(cs_sales_price#83 as decimal(12,2)) AS agg4#104, cast(cs_net_profit#85 as decimal(12,2)) AS agg5#105, cast(c_birth_year#93 as decimal(12,2)) AS agg6#106, cast(cd_dep_count#89 as decimal(12,2)) AS agg7#107] + +(62) ColumnarToRow [codegen id : 3] +Input [10]: [i_item_id#100, ca_country#97, ca_state#96, agg1#101, agg2#102, agg3#103, agg4#104, agg5#105, agg6#106, agg7#107] + +(63) HashAggregate [codegen id : 3] +Input [10]: [i_item_id#100, ca_country#97, ca_state#96, agg1#101, agg2#102, agg3#103, agg4#104, agg5#105, agg6#106, agg7#107] +Keys [3]: [i_item_id#100, ca_country#97, ca_state#96] +Functions [7]: [partial_avg(agg1#101), partial_avg(agg2#102), partial_avg(agg3#103), partial_avg(agg4#104), partial_avg(agg5#105), partial_avg(agg6#106), partial_avg(agg7#107)] +Aggregate Attributes [14]: [sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121] +Results [17]: [i_item_id#100, ca_country#97, ca_state#96, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] + +(64) Exchange +Input [17]: [i_item_id#100, ca_country#97, ca_state#96, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Arguments: hashpartitioning(i_item_id#100, ca_country#97, ca_state#96, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(65) HashAggregate [codegen id : 4] +Input [17]: [i_item_id#100, ca_country#97, ca_state#96, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Keys [3]: [i_item_id#100, ca_country#97, ca_state#96] +Functions [7]: [avg(agg1#101), avg(agg2#102), avg(agg3#103), avg(agg4#104), avg(agg5#105), avg(agg6#106), avg(agg7#107)] +Aggregate Attributes [7]: [avg(agg1#101)#136, avg(agg2#102)#137, avg(agg3#103)#138, avg(agg4#104)#139, avg(agg5#105)#140, avg(agg6#106)#141, avg(agg7#107)#142] +Results [11]: [i_item_id#100, ca_country#97, ca_state#96, null AS county#143, avg(agg1#101)#136 AS agg1#144, avg(agg2#102)#137 AS agg2#145, avg(agg3#103)#138 AS agg3#146, avg(agg4#104)#139 AS agg4#147, avg(agg5#105)#140 AS agg5#148, avg(agg6#106)#141 AS agg6#149, avg(agg7#107)#142 AS agg7#150] + +(66) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#151, cs_bill_cdemo_sk#152, cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#158), dynamicpruningexpression(cs_sold_date_sk#158 IN dynamicpruning#159)] +PartitionFilters: [isnotnull(cs_sold_date_sk#159), dynamicpruningexpression(cs_sold_date_sk#159 IN dynamicpruning#160)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(70) CometFilter -Input [9]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158] -Condition : ((isnotnull(cs_bill_cdemo_sk#151) AND isnotnull(cs_bill_customer_sk#150)) AND isnotnull(cs_item_sk#152)) - -(71) ColumnarToRow [codegen id : 23] -Input [9]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158] +(67) CometFilter +Input [9]: [cs_bill_customer_sk#151, cs_bill_cdemo_sk#152, cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159] +Condition : ((isnotnull(cs_bill_cdemo_sk#152) AND isnotnull(cs_bill_customer_sk#151)) AND isnotnull(cs_item_sk#153)) -(72) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#160, cd_dep_count#161] +(68) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#161, cd_dep_count#162] -(73) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#151] -Right keys [1]: [cd_demo_sk#160] -Join type: Inner -Join condition: None +(69) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#151, cs_bill_cdemo_sk#152, cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159] +Right output [2]: [cd_demo_sk#161, cd_dep_count#162] +Arguments: [cs_bill_cdemo_sk#152], [cd_demo_sk#161], Inner, BuildRight -(74) Project [codegen id : 23] -Output [9]: [cs_bill_customer_sk#150, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161] -Input [11]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_demo_sk#160, cd_dep_count#161] +(70) CometProject +Input [11]: [cs_bill_customer_sk#151, cs_bill_cdemo_sk#152, cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_demo_sk#161, cd_dep_count#162] +Arguments: [cs_bill_customer_sk#151, cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162], [cs_bill_customer_sk#151, cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162] -(75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#162, c_current_cdemo_sk#163, c_current_addr_sk#164, c_birth_year#165] +(71) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] -(76) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#150] -Right keys [1]: [c_customer_sk#162] -Join type: Inner -Join condition: None +(72) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#151, cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162] +Right output [4]: [c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] +Arguments: [cs_bill_customer_sk#151], [c_customer_sk#163], Inner, BuildRight -(77) Project [codegen id : 23] -Output [11]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_current_cdemo_sk#163, c_current_addr_sk#164, c_birth_year#165] -Input [13]: [cs_bill_customer_sk#150, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_customer_sk#162, c_current_cdemo_sk#163, c_current_addr_sk#164, c_birth_year#165] +(73) CometProject +Input [13]: [cs_bill_customer_sk#151, cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] +Arguments: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166], [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] -(78) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#166] +(74) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#167] -(79) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#163] -Right keys [1]: [cd_demo_sk#166] -Join type: Inner -Join condition: None +(75) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] +Right output [1]: [cd_demo_sk#167] +Arguments: [c_current_cdemo_sk#164], [cd_demo_sk#167], Inner, BuildRight -(80) Project [codegen id : 23] -Output [10]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_current_addr_sk#164, c_birth_year#165] -Input [12]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_current_cdemo_sk#163, c_current_addr_sk#164, c_birth_year#165, cd_demo_sk#166] +(76) CometProject +Input [12]: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166, cd_demo_sk#167] +Arguments: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166], [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166] -(81) Scan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#167, ca_state#168, ca_country#169] +(77) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#168, ca_state#169, ca_country#170] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(82) CometFilter -Input [3]: [ca_address_sk#167, ca_state#168, ca_country#169] -Condition : (ca_state#168 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#167)) - -(83) CometProject -Input [3]: [ca_address_sk#167, ca_state#168, ca_country#169] -Arguments: [ca_address_sk#167, ca_country#169], [ca_address_sk#167, ca_country#169] - -(84) ColumnarToRow [codegen id : 20] -Input [2]: [ca_address_sk#167, ca_country#169] - -(85) BroadcastExchange -Input [2]: [ca_address_sk#167, ca_country#169] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(86) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#164] -Right keys [1]: [ca_address_sk#167] -Join type: Inner -Join condition: None - -(87) Project [codegen id : 23] -Output [10]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_birth_year#165, ca_country#169] -Input [12]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_current_addr_sk#164, c_birth_year#165, ca_address_sk#167, ca_country#169] - -(88) ReusedExchange [Reuses operator id: 158] -Output [1]: [d_date_sk#170] - -(89) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#158] -Right keys [1]: [d_date_sk#170] -Join type: Inner -Join condition: None - -(90) Project [codegen id : 23] -Output [9]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cd_dep_count#161, c_birth_year#165, ca_country#169] -Input [11]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_birth_year#165, ca_country#169, d_date_sk#170] - -(91) ReusedExchange [Reuses operator id: 36] -Output [2]: [i_item_sk#171, i_item_id#172] - -(92) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#152] -Right keys [1]: [i_item_sk#171] -Join type: Inner -Join condition: None - -(93) Project [codegen id : 23] -Output [9]: [i_item_id#172, ca_country#169, cast(cs_quantity#153 as decimal(12,2)) AS agg1#173, cast(cs_list_price#154 as decimal(12,2)) AS agg2#174, cast(cs_coupon_amt#156 as decimal(12,2)) AS agg3#175, cast(cs_sales_price#155 as decimal(12,2)) AS agg4#176, cast(cs_net_profit#157 as decimal(12,2)) AS agg5#177, cast(c_birth_year#165 as decimal(12,2)) AS agg6#178, cast(cd_dep_count#161 as decimal(12,2)) AS agg7#179] -Input [11]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cd_dep_count#161, c_birth_year#165, ca_country#169, i_item_sk#171, i_item_id#172] - -(94) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#172, ca_country#169, agg1#173, agg2#174, agg3#175, agg4#176, agg5#177, agg6#178, agg7#179] -Keys [2]: [i_item_id#172, ca_country#169] -Functions [7]: [partial_avg(agg1#173), partial_avg(agg2#174), partial_avg(agg3#175), partial_avg(agg4#176), partial_avg(agg5#177), partial_avg(agg6#178), partial_avg(agg7#179)] -Aggregate Attributes [14]: [sum#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193] -Results [16]: [i_item_id#172, ca_country#169, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] - -(95) Exchange -Input [16]: [i_item_id#172, ca_country#169, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] -Arguments: hashpartitioning(i_item_id#172, ca_country#169, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(96) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#172, ca_country#169, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] -Keys [2]: [i_item_id#172, ca_country#169] -Functions [7]: [avg(agg1#173), avg(agg2#174), avg(agg3#175), avg(agg4#176), avg(agg5#177), avg(agg6#178), avg(agg7#179)] -Aggregate Attributes [7]: [avg(agg1#173)#208, avg(agg2#174)#209, avg(agg3#175)#210, avg(agg4#176)#211, avg(agg5#177)#212, avg(agg6#178)#213, avg(agg7#179)#214] -Results [11]: [i_item_id#172, ca_country#169, null AS ca_state#215, null AS county#216, avg(agg1#173)#208 AS agg1#217, avg(agg2#174)#209 AS agg2#218, avg(agg3#175)#210 AS agg3#219, avg(agg4#176)#211 AS agg4#220, avg(agg5#177)#212 AS agg5#221, avg(agg6#178)#213 AS agg6#222, avg(agg7#179)#214 AS agg7#223] - -(97) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#224, cs_bill_cdemo_sk#225, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232] +(78) CometFilter +Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] +Condition : (ca_state#169 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#168)) + +(79) CometProject +Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] +Arguments: [ca_address_sk#168, ca_country#170], [ca_address_sk#168, ca_country#170] + +(80) CometBroadcastExchange +Input [2]: [ca_address_sk#168, ca_country#170] +Arguments: [ca_address_sk#168, ca_country#170] + +(81) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166] +Right output [2]: [ca_address_sk#168, ca_country#170] +Arguments: [c_current_addr_sk#165], [ca_address_sk#168], Inner, BuildRight + +(82) CometProject +Input [12]: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166, ca_address_sk#168, ca_country#170] +Arguments: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_birth_year#166, ca_country#170], [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_birth_year#166, ca_country#170] + +(83) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#171] + +(84) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_birth_year#166, ca_country#170] +Right output [1]: [d_date_sk#171] +Arguments: [cs_sold_date_sk#159], [d_date_sk#171], Inner, BuildRight + +(85) CometProject +Input [11]: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cs_sold_date_sk#159, cd_dep_count#162, c_birth_year#166, ca_country#170, d_date_sk#171] +Arguments: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cd_dep_count#162, c_birth_year#166, ca_country#170], [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cd_dep_count#162, c_birth_year#166, ca_country#170] + +(86) ReusedExchange [Reuses operator id: 33] +Output [2]: [i_item_sk#172, i_item_id#173] + +(87) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cd_dep_count#162, c_birth_year#166, ca_country#170] +Right output [2]: [i_item_sk#172, i_item_id#173] +Arguments: [cs_item_sk#153], [i_item_sk#172], Inner, BuildRight + +(88) CometProject +Input [11]: [cs_item_sk#153, cs_quantity#154, cs_list_price#155, cs_sales_price#156, cs_coupon_amt#157, cs_net_profit#158, cd_dep_count#162, c_birth_year#166, ca_country#170, i_item_sk#172, i_item_id#173] +Arguments: [i_item_id#173, ca_country#170, agg1#174, agg2#175, agg3#176, agg4#177, agg5#178, agg6#179, agg7#180], [i_item_id#173, ca_country#170, cast(cs_quantity#154 as decimal(12,2)) AS agg1#174, cast(cs_list_price#155 as decimal(12,2)) AS agg2#175, cast(cs_coupon_amt#157 as decimal(12,2)) AS agg3#176, cast(cs_sales_price#156 as decimal(12,2)) AS agg4#177, cast(cs_net_profit#158 as decimal(12,2)) AS agg5#178, cast(c_birth_year#166 as decimal(12,2)) AS agg6#179, cast(cd_dep_count#162 as decimal(12,2)) AS agg7#180] + +(89) ColumnarToRow [codegen id : 5] +Input [9]: [i_item_id#173, ca_country#170, agg1#174, agg2#175, agg3#176, agg4#177, agg5#178, agg6#179, agg7#180] + +(90) HashAggregate [codegen id : 5] +Input [9]: [i_item_id#173, ca_country#170, agg1#174, agg2#175, agg3#176, agg4#177, agg5#178, agg6#179, agg7#180] +Keys [2]: [i_item_id#173, ca_country#170] +Functions [7]: [partial_avg(agg1#174), partial_avg(agg2#175), partial_avg(agg3#176), partial_avg(agg4#177), partial_avg(agg5#178), partial_avg(agg6#179), partial_avg(agg7#180)] +Aggregate Attributes [14]: [sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] +Results [16]: [i_item_id#173, ca_country#170, sum#195, count#196, sum#197, count#198, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208] + +(91) Exchange +Input [16]: [i_item_id#173, ca_country#170, sum#195, count#196, sum#197, count#198, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208] +Arguments: hashpartitioning(i_item_id#173, ca_country#170, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(92) HashAggregate [codegen id : 6] +Input [16]: [i_item_id#173, ca_country#170, sum#195, count#196, sum#197, count#198, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208] +Keys [2]: [i_item_id#173, ca_country#170] +Functions [7]: [avg(agg1#174), avg(agg2#175), avg(agg3#176), avg(agg4#177), avg(agg5#178), avg(agg6#179), avg(agg7#180)] +Aggregate Attributes [7]: [avg(agg1#174)#209, avg(agg2#175)#210, avg(agg3#176)#211, avg(agg4#177)#212, avg(agg5#178)#213, avg(agg6#179)#214, avg(agg7#180)#215] +Results [11]: [i_item_id#173, ca_country#170, null AS ca_state#216, null AS county#217, avg(agg1#174)#209 AS agg1#218, avg(agg2#175)#210 AS agg2#219, avg(agg3#176)#211 AS agg3#220, avg(agg4#177)#212 AS agg4#221, avg(agg5#178)#213 AS agg5#222, avg(agg6#179)#214 AS agg6#223, avg(agg7#180)#215 AS agg7#224] + +(93) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#225, cs_bill_cdemo_sk#226, cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#232), dynamicpruningexpression(cs_sold_date_sk#232 IN dynamicpruning#233)] +PartitionFilters: [isnotnull(cs_sold_date_sk#233), dynamicpruningexpression(cs_sold_date_sk#233 IN dynamicpruning#234)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(98) CometFilter -Input [9]: [cs_bill_customer_sk#224, cs_bill_cdemo_sk#225, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232] -Condition : ((isnotnull(cs_bill_cdemo_sk#225) AND isnotnull(cs_bill_customer_sk#224)) AND isnotnull(cs_item_sk#226)) +(94) CometFilter +Input [9]: [cs_bill_customer_sk#225, cs_bill_cdemo_sk#226, cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233] +Condition : ((isnotnull(cs_bill_cdemo_sk#226) AND isnotnull(cs_bill_customer_sk#225)) AND isnotnull(cs_item_sk#227)) -(99) ColumnarToRow [codegen id : 31] -Input [9]: [cs_bill_customer_sk#224, cs_bill_cdemo_sk#225, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232] +(95) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#235, cd_dep_count#236] -(100) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#234, cd_dep_count#235] +(96) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#225, cs_bill_cdemo_sk#226, cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233] +Right output [2]: [cd_demo_sk#235, cd_dep_count#236] +Arguments: [cs_bill_cdemo_sk#226], [cd_demo_sk#235], Inner, BuildRight -(101) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#225] -Right keys [1]: [cd_demo_sk#234] -Join type: Inner -Join condition: None +(97) CometProject +Input [11]: [cs_bill_customer_sk#225, cs_bill_cdemo_sk#226, cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_demo_sk#235, cd_dep_count#236] +Arguments: [cs_bill_customer_sk#225, cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236], [cs_bill_customer_sk#225, cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236] -(102) Project [codegen id : 31] -Output [9]: [cs_bill_customer_sk#224, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235] -Input [11]: [cs_bill_customer_sk#224, cs_bill_cdemo_sk#225, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_demo_sk#234, cd_dep_count#235] +(98) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#237, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240] -(103) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#236, c_current_cdemo_sk#237, c_current_addr_sk#238, c_birth_year#239] +(99) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#225, cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236] +Right output [4]: [c_customer_sk#237, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240] +Arguments: [cs_bill_customer_sk#225], [c_customer_sk#237], Inner, BuildRight -(104) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#224] -Right keys [1]: [c_customer_sk#236] -Join type: Inner -Join condition: None +(100) CometProject +Input [13]: [cs_bill_customer_sk#225, cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_customer_sk#237, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240] +Arguments: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240], [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240] -(105) Project [codegen id : 31] -Output [11]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_current_cdemo_sk#237, c_current_addr_sk#238, c_birth_year#239] -Input [13]: [cs_bill_customer_sk#224, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_customer_sk#236, c_current_cdemo_sk#237, c_current_addr_sk#238, c_birth_year#239] +(101) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#241] -(106) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#240] +(102) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240] +Right output [1]: [cd_demo_sk#241] +Arguments: [c_current_cdemo_sk#238], [cd_demo_sk#241], Inner, BuildRight -(107) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#237] -Right keys [1]: [cd_demo_sk#240] -Join type: Inner -Join condition: None +(103) CometProject +Input [12]: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240, cd_demo_sk#241] +Arguments: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_current_addr_sk#239, c_birth_year#240], [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_current_addr_sk#239, c_birth_year#240] -(108) Project [codegen id : 31] -Output [10]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_current_addr_sk#238, c_birth_year#239] -Input [12]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_current_cdemo_sk#237, c_current_addr_sk#238, c_birth_year#239, cd_demo_sk#240] - -(109) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#241, ca_state#242] +(104) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#242, ca_state#243] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(110) CometFilter -Input [2]: [ca_address_sk#241, ca_state#242] -Condition : (ca_state#242 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#241)) - -(111) CometProject -Input [2]: [ca_address_sk#241, ca_state#242] -Arguments: [ca_address_sk#241], [ca_address_sk#241] - -(112) ColumnarToRow [codegen id : 28] -Input [1]: [ca_address_sk#241] - -(113) BroadcastExchange -Input [1]: [ca_address_sk#241] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(114) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#238] -Right keys [1]: [ca_address_sk#241] -Join type: Inner -Join condition: None - -(115) Project [codegen id : 31] -Output [9]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_birth_year#239] -Input [11]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_current_addr_sk#238, c_birth_year#239, ca_address_sk#241] - -(116) ReusedExchange [Reuses operator id: 158] -Output [1]: [d_date_sk#243] - -(117) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#232] -Right keys [1]: [d_date_sk#243] -Join type: Inner -Join condition: None - -(118) Project [codegen id : 31] -Output [8]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cd_dep_count#235, c_birth_year#239] -Input [10]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_birth_year#239, d_date_sk#243] - -(119) ReusedExchange [Reuses operator id: 36] -Output [2]: [i_item_sk#244, i_item_id#245] - -(120) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#226] -Right keys [1]: [i_item_sk#244] -Join type: Inner -Join condition: None - -(121) Project [codegen id : 31] -Output [8]: [i_item_id#245, cast(cs_quantity#227 as decimal(12,2)) AS agg1#246, cast(cs_list_price#228 as decimal(12,2)) AS agg2#247, cast(cs_coupon_amt#230 as decimal(12,2)) AS agg3#248, cast(cs_sales_price#229 as decimal(12,2)) AS agg4#249, cast(cs_net_profit#231 as decimal(12,2)) AS agg5#250, cast(c_birth_year#239 as decimal(12,2)) AS agg6#251, cast(cd_dep_count#235 as decimal(12,2)) AS agg7#252] -Input [10]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cd_dep_count#235, c_birth_year#239, i_item_sk#244, i_item_id#245] - -(122) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#245, agg1#246, agg2#247, agg3#248, agg4#249, agg5#250, agg6#251, agg7#252] -Keys [1]: [i_item_id#245] -Functions [7]: [partial_avg(agg1#246), partial_avg(agg2#247), partial_avg(agg3#248), partial_avg(agg4#249), partial_avg(agg5#250), partial_avg(agg6#251), partial_avg(agg7#252)] -Aggregate Attributes [14]: [sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266] -Results [15]: [i_item_id#245, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] - -(123) Exchange -Input [15]: [i_item_id#245, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] -Arguments: hashpartitioning(i_item_id#245, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(124) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#245, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] -Keys [1]: [i_item_id#245] -Functions [7]: [avg(agg1#246), avg(agg2#247), avg(agg3#248), avg(agg4#249), avg(agg5#250), avg(agg6#251), avg(agg7#252)] -Aggregate Attributes [7]: [avg(agg1#246)#281, avg(agg2#247)#282, avg(agg3#248)#283, avg(agg4#249)#284, avg(agg5#250)#285, avg(agg6#251)#286, avg(agg7#252)#287] -Results [11]: [i_item_id#245, null AS ca_country#288, null AS ca_state#289, null AS county#290, avg(agg1#246)#281 AS agg1#291, avg(agg2#247)#282 AS agg2#292, avg(agg3#248)#283 AS agg3#293, avg(agg4#249)#284 AS agg4#294, avg(agg5#250)#285 AS agg5#295, avg(agg6#251)#286 AS agg6#296, avg(agg7#252)#287 AS agg7#297] - -(125) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306] +(105) CometFilter +Input [2]: [ca_address_sk#242, ca_state#243] +Condition : (ca_state#243 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#242)) + +(106) CometProject +Input [2]: [ca_address_sk#242, ca_state#243] +Arguments: [ca_address_sk#242], [ca_address_sk#242] + +(107) CometBroadcastExchange +Input [1]: [ca_address_sk#242] +Arguments: [ca_address_sk#242] + +(108) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_current_addr_sk#239, c_birth_year#240] +Right output [1]: [ca_address_sk#242] +Arguments: [c_current_addr_sk#239], [ca_address_sk#242], Inner, BuildRight + +(109) CometProject +Input [11]: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_current_addr_sk#239, c_birth_year#240, ca_address_sk#242] +Arguments: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_birth_year#240], [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_birth_year#240] + +(110) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#244] + +(111) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_birth_year#240] +Right output [1]: [d_date_sk#244] +Arguments: [cs_sold_date_sk#233], [d_date_sk#244], Inner, BuildRight + +(112) CometProject +Input [10]: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cs_sold_date_sk#233, cd_dep_count#236, c_birth_year#240, d_date_sk#244] +Arguments: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cd_dep_count#236, c_birth_year#240], [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cd_dep_count#236, c_birth_year#240] + +(113) ReusedExchange [Reuses operator id: 33] +Output [2]: [i_item_sk#245, i_item_id#246] + +(114) CometBroadcastHashJoin +Left output [8]: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cd_dep_count#236, c_birth_year#240] +Right output [2]: [i_item_sk#245, i_item_id#246] +Arguments: [cs_item_sk#227], [i_item_sk#245], Inner, BuildRight + +(115) CometProject +Input [10]: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, cd_dep_count#236, c_birth_year#240, i_item_sk#245, i_item_id#246] +Arguments: [i_item_id#246, agg1#247, agg2#248, agg3#249, agg4#250, agg5#251, agg6#252, agg7#253], [i_item_id#246, cast(cs_quantity#228 as decimal(12,2)) AS agg1#247, cast(cs_list_price#229 as decimal(12,2)) AS agg2#248, cast(cs_coupon_amt#231 as decimal(12,2)) AS agg3#249, cast(cs_sales_price#230 as decimal(12,2)) AS agg4#250, cast(cs_net_profit#232 as decimal(12,2)) AS agg5#251, cast(c_birth_year#240 as decimal(12,2)) AS agg6#252, cast(cd_dep_count#236 as decimal(12,2)) AS agg7#253] + +(116) ColumnarToRow [codegen id : 7] +Input [8]: [i_item_id#246, agg1#247, agg2#248, agg3#249, agg4#250, agg5#251, agg6#252, agg7#253] + +(117) HashAggregate [codegen id : 7] +Input [8]: [i_item_id#246, agg1#247, agg2#248, agg3#249, agg4#250, agg5#251, agg6#252, agg7#253] +Keys [1]: [i_item_id#246] +Functions [7]: [partial_avg(agg1#247), partial_avg(agg2#248), partial_avg(agg3#249), partial_avg(agg4#250), partial_avg(agg5#251), partial_avg(agg6#252), partial_avg(agg7#253)] +Aggregate Attributes [14]: [sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267] +Results [15]: [i_item_id#246, sum#268, count#269, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281] + +(118) Exchange +Input [15]: [i_item_id#246, sum#268, count#269, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281] +Arguments: hashpartitioning(i_item_id#246, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(119) HashAggregate [codegen id : 8] +Input [15]: [i_item_id#246, sum#268, count#269, sum#270, count#271, sum#272, count#273, sum#274, count#275, sum#276, count#277, sum#278, count#279, sum#280, count#281] +Keys [1]: [i_item_id#246] +Functions [7]: [avg(agg1#247), avg(agg2#248), avg(agg3#249), avg(agg4#250), avg(agg5#251), avg(agg6#252), avg(agg7#253)] +Aggregate Attributes [7]: [avg(agg1#247)#282, avg(agg2#248)#283, avg(agg3#249)#284, avg(agg4#250)#285, avg(agg5#251)#286, avg(agg6#252)#287, avg(agg7#253)#288] +Results [11]: [i_item_id#246, null AS ca_country#289, null AS ca_state#290, null AS county#291, avg(agg1#247)#282 AS agg1#292, avg(agg2#248)#283 AS agg2#293, avg(agg3#249)#284 AS agg3#294, avg(agg4#250)#285 AS agg4#295, avg(agg5#251)#286 AS agg5#296, avg(agg6#252)#287 AS agg6#297, avg(agg7#253)#288 AS agg7#298] + +(120) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#299, cs_bill_cdemo_sk#300, cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#306), dynamicpruningexpression(cs_sold_date_sk#306 IN dynamicpruning#307)] +PartitionFilters: [isnotnull(cs_sold_date_sk#307), dynamicpruningexpression(cs_sold_date_sk#307 IN dynamicpruning#308)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(126) CometFilter -Input [9]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306] -Condition : ((isnotnull(cs_bill_cdemo_sk#299) AND isnotnull(cs_bill_customer_sk#298)) AND isnotnull(cs_item_sk#300)) - -(127) ColumnarToRow [codegen id : 39] -Input [9]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306] +(121) CometFilter +Input [9]: [cs_bill_customer_sk#299, cs_bill_cdemo_sk#300, cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307] +Condition : ((isnotnull(cs_bill_cdemo_sk#300) AND isnotnull(cs_bill_customer_sk#299)) AND isnotnull(cs_item_sk#301)) -(128) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#308, cd_dep_count#309] +(122) ReusedExchange [Reuses operator id: 6] +Output [2]: [cd_demo_sk#309, cd_dep_count#310] -(129) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#299] -Right keys [1]: [cd_demo_sk#308] -Join type: Inner -Join condition: None +(123) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#299, cs_bill_cdemo_sk#300, cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307] +Right output [2]: [cd_demo_sk#309, cd_dep_count#310] +Arguments: [cs_bill_cdemo_sk#300], [cd_demo_sk#309], Inner, BuildRight -(130) Project [codegen id : 39] -Output [9]: [cs_bill_customer_sk#298, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309] -Input [11]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_demo_sk#308, cd_dep_count#309] +(124) CometProject +Input [11]: [cs_bill_customer_sk#299, cs_bill_cdemo_sk#300, cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_demo_sk#309, cd_dep_count#310] +Arguments: [cs_bill_customer_sk#299, cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310], [cs_bill_customer_sk#299, cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310] -(131) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#310, c_current_cdemo_sk#311, c_current_addr_sk#312, c_birth_year#313] +(125) ReusedExchange [Reuses operator id: 12] +Output [4]: [c_customer_sk#311, c_current_cdemo_sk#312, c_current_addr_sk#313, c_birth_year#314] -(132) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#298] -Right keys [1]: [c_customer_sk#310] -Join type: Inner -Join condition: None +(126) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#299, cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310] +Right output [4]: [c_customer_sk#311, c_current_cdemo_sk#312, c_current_addr_sk#313, c_birth_year#314] +Arguments: [cs_bill_customer_sk#299], [c_customer_sk#311], Inner, BuildRight -(133) Project [codegen id : 39] -Output [11]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_current_cdemo_sk#311, c_current_addr_sk#312, c_birth_year#313] -Input [13]: [cs_bill_customer_sk#298, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_customer_sk#310, c_current_cdemo_sk#311, c_current_addr_sk#312, c_birth_year#313] +(127) CometProject +Input [13]: [cs_bill_customer_sk#299, cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_customer_sk#311, c_current_cdemo_sk#312, c_current_addr_sk#313, c_birth_year#314] +Arguments: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_current_cdemo_sk#312, c_current_addr_sk#313, c_birth_year#314], [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_current_cdemo_sk#312, c_current_addr_sk#313, c_birth_year#314] -(134) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#314] +(128) ReusedExchange [Reuses operator id: 17] +Output [1]: [cd_demo_sk#315] -(135) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#311] -Right keys [1]: [cd_demo_sk#314] -Join type: Inner -Join condition: None +(129) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_current_cdemo_sk#312, c_current_addr_sk#313, c_birth_year#314] +Right output [1]: [cd_demo_sk#315] +Arguments: [c_current_cdemo_sk#312], [cd_demo_sk#315], Inner, BuildRight -(136) Project [codegen id : 39] -Output [10]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_current_addr_sk#312, c_birth_year#313] -Input [12]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_current_cdemo_sk#311, c_current_addr_sk#312, c_birth_year#313, cd_demo_sk#314] +(130) CometProject +Input [12]: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_current_cdemo_sk#312, c_current_addr_sk#313, c_birth_year#314, cd_demo_sk#315] +Arguments: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_current_addr_sk#313, c_birth_year#314], [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_current_addr_sk#313, c_birth_year#314] -(137) ReusedExchange [Reuses operator id: 113] -Output [1]: [ca_address_sk#315] +(131) ReusedExchange [Reuses operator id: 107] +Output [1]: [ca_address_sk#316] -(138) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#312] -Right keys [1]: [ca_address_sk#315] -Join type: Inner -Join condition: None +(132) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_current_addr_sk#313, c_birth_year#314] +Right output [1]: [ca_address_sk#316] +Arguments: [c_current_addr_sk#313], [ca_address_sk#316], Inner, BuildRight -(139) Project [codegen id : 39] -Output [9]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_birth_year#313] -Input [11]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_current_addr_sk#312, c_birth_year#313, ca_address_sk#315] +(133) CometProject +Input [11]: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_current_addr_sk#313, c_birth_year#314, ca_address_sk#316] +Arguments: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_birth_year#314], [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_birth_year#314] -(140) ReusedExchange [Reuses operator id: 158] -Output [1]: [d_date_sk#316] +(134) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#317] -(141) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#306] -Right keys [1]: [d_date_sk#316] -Join type: Inner -Join condition: None +(135) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_birth_year#314] +Right output [1]: [d_date_sk#317] +Arguments: [cs_sold_date_sk#307], [d_date_sk#317], Inner, BuildRight -(142) Project [codegen id : 39] -Output [8]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cd_dep_count#309, c_birth_year#313] -Input [10]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_birth_year#313, d_date_sk#316] +(136) CometProject +Input [10]: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cs_sold_date_sk#307, cd_dep_count#310, c_birth_year#314, d_date_sk#317] +Arguments: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cd_dep_count#310, c_birth_year#314], [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cd_dep_count#310, c_birth_year#314] -(143) Scan parquet spark_catalog.default.item -Output [1]: [i_item_sk#317] +(137) Scan parquet spark_catalog.default.item +Output [1]: [i_item_sk#318] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(144) CometFilter -Input [1]: [i_item_sk#317] -Condition : isnotnull(i_item_sk#317) +(138) CometFilter +Input [1]: [i_item_sk#318] +Condition : isnotnull(i_item_sk#318) -(145) ColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#317] +(139) CometBroadcastExchange +Input [1]: [i_item_sk#318] +Arguments: [i_item_sk#318] -(146) BroadcastExchange -Input [1]: [i_item_sk#317] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +(140) CometBroadcastHashJoin +Left output [8]: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cd_dep_count#310, c_birth_year#314] +Right output [1]: [i_item_sk#318] +Arguments: [cs_item_sk#301], [i_item_sk#318], Inner, BuildRight -(147) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#300] -Right keys [1]: [i_item_sk#317] -Join type: Inner -Join condition: None +(141) CometProject +Input [9]: [cs_item_sk#301, cs_quantity#302, cs_list_price#303, cs_sales_price#304, cs_coupon_amt#305, cs_net_profit#306, cd_dep_count#310, c_birth_year#314, i_item_sk#318] +Arguments: [agg1#319, agg2#320, agg3#321, agg4#322, agg5#323, agg6#324, agg7#325], [cast(cs_quantity#302 as decimal(12,2)) AS agg1#319, cast(cs_list_price#303 as decimal(12,2)) AS agg2#320, cast(cs_coupon_amt#305 as decimal(12,2)) AS agg3#321, cast(cs_sales_price#304 as decimal(12,2)) AS agg4#322, cast(cs_net_profit#306 as decimal(12,2)) AS agg5#323, cast(c_birth_year#314 as decimal(12,2)) AS agg6#324, cast(cd_dep_count#310 as decimal(12,2)) AS agg7#325] -(148) Project [codegen id : 39] -Output [7]: [cast(cs_quantity#301 as decimal(12,2)) AS agg1#318, cast(cs_list_price#302 as decimal(12,2)) AS agg2#319, cast(cs_coupon_amt#304 as decimal(12,2)) AS agg3#320, cast(cs_sales_price#303 as decimal(12,2)) AS agg4#321, cast(cs_net_profit#305 as decimal(12,2)) AS agg5#322, cast(c_birth_year#313 as decimal(12,2)) AS agg6#323, cast(cd_dep_count#309 as decimal(12,2)) AS agg7#324] -Input [9]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cd_dep_count#309, c_birth_year#313, i_item_sk#317] +(142) ColumnarToRow [codegen id : 9] +Input [7]: [agg1#319, agg2#320, agg3#321, agg4#322, agg5#323, agg6#324, agg7#325] -(149) HashAggregate [codegen id : 39] -Input [7]: [agg1#318, agg2#319, agg3#320, agg4#321, agg5#322, agg6#323, agg7#324] +(143) HashAggregate [codegen id : 9] +Input [7]: [agg1#319, agg2#320, agg3#321, agg4#322, agg5#323, agg6#324, agg7#325] Keys: [] -Functions [7]: [partial_avg(agg1#318), partial_avg(agg2#319), partial_avg(agg3#320), partial_avg(agg4#321), partial_avg(agg5#322), partial_avg(agg6#323), partial_avg(agg7#324)] -Aggregate Attributes [14]: [sum#325, count#326, sum#327, count#328, sum#329, count#330, sum#331, count#332, sum#333, count#334, sum#335, count#336, sum#337, count#338] -Results [14]: [sum#339, count#340, sum#341, count#342, sum#343, count#344, sum#345, count#346, sum#347, count#348, sum#349, count#350, sum#351, count#352] +Functions [7]: [partial_avg(agg1#319), partial_avg(agg2#320), partial_avg(agg3#321), partial_avg(agg4#322), partial_avg(agg5#323), partial_avg(agg6#324), partial_avg(agg7#325)] +Aggregate Attributes [14]: [sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335, sum#336, count#337, sum#338, count#339] +Results [14]: [sum#340, count#341, sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353] -(150) Exchange -Input [14]: [sum#339, count#340, sum#341, count#342, sum#343, count#344, sum#345, count#346, sum#347, count#348, sum#349, count#350, sum#351, count#352] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] +(144) Exchange +Input [14]: [sum#340, count#341, sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(151) HashAggregate [codegen id : 40] -Input [14]: [sum#339, count#340, sum#341, count#342, sum#343, count#344, sum#345, count#346, sum#347, count#348, sum#349, count#350, sum#351, count#352] +(145) HashAggregate [codegen id : 10] +Input [14]: [sum#340, count#341, sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353] Keys: [] -Functions [7]: [avg(agg1#318), avg(agg2#319), avg(agg3#320), avg(agg4#321), avg(agg5#322), avg(agg6#323), avg(agg7#324)] -Aggregate Attributes [7]: [avg(agg1#318)#353, avg(agg2#319)#354, avg(agg3#320)#355, avg(agg4#321)#356, avg(agg5#322)#357, avg(agg6#323)#358, avg(agg7#324)#359] -Results [11]: [null AS i_item_id#360, null AS ca_country#361, null AS ca_state#362, null AS county#363, avg(agg1#318)#353 AS agg1#364, avg(agg2#319)#354 AS agg2#365, avg(agg3#320)#355 AS agg3#366, avg(agg4#321)#356 AS agg4#367, avg(agg5#322)#357 AS agg5#368, avg(agg6#323)#358 AS agg6#369, avg(agg7#324)#359 AS agg7#370] +Functions [7]: [avg(agg1#319), avg(agg2#320), avg(agg3#321), avg(agg4#322), avg(agg5#323), avg(agg6#324), avg(agg7#325)] +Aggregate Attributes [7]: [avg(agg1#319)#354, avg(agg2#320)#355, avg(agg3#321)#356, avg(agg4#322)#357, avg(agg5#323)#358, avg(agg6#324)#359, avg(agg7#325)#360] +Results [11]: [null AS i_item_id#361, null AS ca_country#362, null AS ca_state#363, null AS county#364, avg(agg1#319)#354 AS agg1#365, avg(agg2#320)#355 AS agg2#366, avg(agg3#321)#356 AS agg3#367, avg(agg4#322)#357 AS agg4#368, avg(agg5#323)#358 AS agg5#369, avg(agg6#324)#359 AS agg6#370, avg(agg7#325)#360 AS agg7#371] -(152) Union +(146) Union -(153) TakeOrderedAndProject -Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] -Arguments: 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#27 ASC NULLS FIRST], [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] +(147) TakeOrderedAndProject +Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#71, agg2#72, agg3#73, agg4#74, agg5#75, agg6#76, agg7#77] +Arguments: 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#71, agg2#72, agg3#73, agg4#74, agg5#75, agg6#76, agg7#77] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (158) -+- * ColumnarToRow (157) - +- CometProject (156) - +- CometFilter (155) - +- CometScan parquet spark_catalog.default.date_dim (154) +BroadcastExchange (152) ++- * ColumnarToRow (151) + +- CometProject (150) + +- CometFilter (149) + +- CometScan parquet spark_catalog.default.date_dim (148) -(154) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#371] +(148) Scan parquet spark_catalog.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,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(155) CometFilter -Input [2]: [d_date_sk#25, d_year#371] -Condition : ((isnotnull(d_year#371) AND (d_year#371 = 2001)) AND isnotnull(d_date_sk#25)) +(149) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#25)) -(156) CometProject -Input [2]: [d_date_sk#25, d_year#371] +(150) CometProject +Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(157) ColumnarToRow [codegen id : 1] +(151) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(158) BroadcastExchange +(152) BroadcastExchange Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 42 Hosting Expression = cs_sold_date_sk#85 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#158 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#159 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 97 Hosting Expression = cs_sold_date_sk#232 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#233 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#306 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 120 Hosting Expression = cs_sold_date_sk#307 IN dynamicpruning#10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt index f02809572c..9f43cdc72f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] Union - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [i_item_id,ca_country,ca_state,ca_county,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),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,ca_country,ca_state,ca_county] #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -30,204 +30,150 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (16) + CometBroadcastExchange #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange #5 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange #6 + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange #7 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #8 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (4) 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 - Exchange [i_item_id,ca_country,ca_state] #8 - WholeStageCodegen (15) + Exchange [i_item_id,ca_country,ca_state] #9 + WholeStageCodegen (3) HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (24) + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange #10 + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + WholeStageCodegen (6) 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 - Exchange [i_item_id,ca_country] #10 - WholeStageCodegen (23) + Exchange [i_item_id,ca_country] #11 + WholeStageCodegen (5) HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (32) + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange #12 + CometProject [ca_address_sk,ca_country] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + WholeStageCodegen (8) 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 - Exchange [i_item_id] #12 - WholeStageCodegen (31) + Exchange [i_item_id] #13 + WholeStageCodegen (7) HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (28) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (40) + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange #14 + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + WholeStageCodegen (10) HashAggregate [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),i_item_id,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 - Exchange #14 - WholeStageCodegen (39) + Exchange #15 + WholeStageCodegen (9) HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #13 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (38) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + ReusedExchange [ca_address_sk] #14 + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange #16 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt index 1304af2e10..6619cc9d51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * HashAggregate (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.catalog_sales @@ -33,118 +35,129 @@ ReadSchema: struct Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [cs_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] +Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(15) HashAggregate [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Aggregate Attributes [1]: [sum#13] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] -(14) Exchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +(17) HashAggregate [codegen id : 2] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] -Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#15] +Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#15,17,2) AS itemrevenue#16, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#15,17,2) AS _w0#17] -(16) Exchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(18) Exchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +(19) Sort [codegen id : 3] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(20) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] +Arguments: [sum(_w0#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9] -(19) Project [codegen id : 6] -Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] -Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] +(21) Project [codegen id : 4] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, ((_w0#17 * 100) / _we0#18) AS revenueratio#19] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, _we0#18] -(20) TakeOrderedAndProject -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +(22) TakeOrderedAndProject +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#19 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) -(21) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(23) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt index 2a2a392cd0..da567f6874 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,12 +29,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt index fdebdc8a4e..2810779ed7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt @@ -1,26 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * Expand (18) - +- * Project (17) - +- * BroadcastNestedLoopJoin Inner BuildRight (16) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometScan parquet spark_catalog.default.warehouse (13) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * Expand (20) + +- * Project (19) + +- * BroadcastNestedLoopJoin Inner BuildRight (18) + :- * ColumnarToRow (14) + : +- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.item (9) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometScan parquet spark_catalog.default.warehouse (15) (1) Scan parquet spark_catalog.default.inventory @@ -35,127 +37,138 @@ ReadSchema: struct Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] Condition : isnotnull(inv_item_sk#1) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 27] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] -(6) Project [codegen id : 4] -Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] +(7) CometBroadcastHashJoin +Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] +Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] -(7) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +(9) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Condition : isnotnull(i_item_sk#6) +(10) CometFilter +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Condition : isnotnull(i_item_sk#7) -(9) ColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +(11) CometBroadcastExchange +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(10) BroadcastExchange -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] +Right output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(13) CometProject +Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11], [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(12) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +(14) ColumnarToRow [codegen id : 2] +Input [5]: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(13) Scan parquet spark_catalog.default.warehouse +(15) Scan parquet spark_catalog.default.warehouse Output: [] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(14) ColumnarToRow [codegen id : 3] +(16) ColumnarToRow [codegen id : 1] Input: [] -(15) BroadcastExchange +(17) BroadcastExchange Input: [] -Arguments: IdentityBroadcastMode, [plan_id=2] +Arguments: IdentityBroadcastMode, [plan_id=1] -(16) BroadcastNestedLoopJoin [codegen id : 4] +(18) BroadcastNestedLoopJoin [codegen id : 2] Join type: Inner Join condition: None -(17) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, i_category#9] -Input [5]: [inv_quantity_on_hand#2, i_brand#7, i_class#8, i_category#9, i_product_name#10] +(19) Project [codegen id : 2] +Output [5]: [inv_quantity_on_hand#2, i_product_name#11, i_brand#8, i_class#9, i_category#10] +Input [5]: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(18) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, i_category#9] -Arguments: [[inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, i_category#9, 0], [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, null, 1], [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, null, null, 3], [inv_quantity_on_hand#2, i_product_name#10, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] +(20) Expand [codegen id : 2] +Input [5]: [inv_quantity_on_hand#2, i_product_name#11, i_brand#8, i_class#9, i_category#10] +Arguments: [[inv_quantity_on_hand#2, i_product_name#11, i_brand#8, i_class#9, i_category#10, 0], [inv_quantity_on_hand#2, i_product_name#11, i_brand#8, i_class#9, null, 1], [inv_quantity_on_hand#2, i_product_name#11, i_brand#8, null, null, 3], [inv_quantity_on_hand#2, i_product_name#11, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16] -(19) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#2, i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] -Keys [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] +(21) HashAggregate [codegen id : 2] +Input [6]: [inv_quantity_on_hand#2, i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16] +Keys [5]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#16, count#17] -Results [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] +Aggregate Attributes [2]: [sum#17, count#18] +Results [7]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16, sum#19, count#20] -(20) Exchange -Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) Exchange +Input [7]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16, sum#19, count#20] +Arguments: hashpartitioning(i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] -Keys [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] +(23) HashAggregate [codegen id : 3] +Input [7]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16, sum#19, count#20] +Keys [5]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16] Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#20] -Results [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, avg(inv_quantity_on_hand#2)#20 AS qoh#21] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#21] +Results [5]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, avg(inv_quantity_on_hand#2)#21 AS qoh#22] -(22) TakeOrderedAndProject -Input [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] -Arguments: 100, [qoh#21 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#12 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_category#14 ASC NULLS FIRST], [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] +(24) TakeOrderedAndProject +Input [5]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, qoh#22] +Arguments: 100, [qoh#22 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_brand#13 ASC NULLS FIRST, i_class#14 ASC NULLS FIRST, i_category#15 ASC NULLS FIRST], [i_product_name#12, i_brand#13, i_class#14, i_category#15, qoh#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) -(23) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#22] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#22] -Condition : (((isnotnull(d_month_seq#22) AND (d_month_seq#22 >= 1200)) AND (d_month_seq#22 <= 1211)) AND isnotnull(d_date_sk#5)) +(26) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(25) CometProject -Input [2]: [d_date_sk#5, d_month_seq#22] +(27) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt index 63a428d4e4..415b430f86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] BroadcastNestedLoopJoin - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometProject [inv_item_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometFilter [inv_item_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -24,18 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt index 1e5f5c2f7b..ac15878d51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt @@ -1,49 +1,50 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- Union (44) - :- * HashAggregate (23) - : +- * HashAggregate (22) - : +- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.warehouse (13) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * HashAggregate (25) - : +- ReusedExchange (24) - :- * HashAggregate (33) - : +- Exchange (32) - : +- * HashAggregate (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) - :- * HashAggregate (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- * HashAggregate (35) - : +- ReusedExchange (34) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * HashAggregate (40) - +- ReusedExchange (39) +TakeOrderedAndProject (46) ++- Union (45) + :- * HashAggregate (24) + : +- * HashAggregate (23) + : +- * HashAggregate (22) + : +- Exchange (21) + : +- * HashAggregate (20) + : +- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.item (9) + : +- CometBroadcastExchange (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.warehouse (14) + :- * HashAggregate (29) + : +- Exchange (28) + : +- * HashAggregate (27) + : +- * HashAggregate (26) + : +- ReusedExchange (25) + :- * HashAggregate (34) + : +- Exchange (33) + : +- * HashAggregate (32) + : +- * HashAggregate (31) + : +- ReusedExchange (30) + :- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * HashAggregate (36) + : +- ReusedExchange (35) + +- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- ReusedExchange (40) (1) Scan parquet spark_catalog.default.inventory @@ -58,258 +59,265 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +(7) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] +Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -(7) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(9) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(9) ColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(10) CometFilter +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Condition : isnotnull(i_item_sk#8) -(10) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +Right output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight -(12) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(13) CometProject +Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(13) Scan parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#12] +(14) Scan parquet spark_catalog.default.warehouse +Output [1]: [w_warehouse_sk#13] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(14) CometFilter -Input [1]: [w_warehouse_sk#12] -Condition : isnotnull(w_warehouse_sk#12) +(15) CometFilter +Input [1]: [w_warehouse_sk#13] +Condition : isnotnull(w_warehouse_sk#13) -(15) ColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#12] +(16) CometBroadcastExchange +Input [1]: [w_warehouse_sk#13] +Arguments: [w_warehouse_sk#13] -(16) BroadcastExchange -Input [1]: [w_warehouse_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Right output [1]: [w_warehouse_sk#13] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#13], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#12] -Join type: Inner -Join condition: None +(18) CometProject +Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12, w_warehouse_sk#13] +Arguments: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12], [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11, w_warehouse_sk#12] +(19) ColumnarToRow [codegen id : 1] +Input [5]: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +(20) HashAggregate [codegen id : 1] +Input [5]: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#13, count#14] -Results [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] +Aggregate Attributes [2]: [sum#14, count#15] +Results [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#16, count#17] -(20) Exchange -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, i_category#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) Exchange +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#16, count#17] +Arguments: hashpartitioning(i_product_name#12, i_brand#9, i_class#10, i_category#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +(22) HashAggregate [codegen id : 2] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#16, count#17] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(inv_quantity_on_hand#3)#17 AS qoh#18] - -(22) HashAggregate [codegen id : 5] -Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#18] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] -Functions [1]: [partial_avg(qoh#18)] -Aggregate Attributes [2]: [sum#19, count#20] -Results [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] - -(23) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] -Functions [1]: [avg(qoh#18)] -Aggregate Attributes [1]: [avg(qoh#18)#23] -Results [5]: [i_product_name#11 AS i_product_name#24, i_brand#8 AS i_brand#25, i_class#9 AS i_class#26, i_category#10 AS i_category#27, avg(qoh#18)#23 AS qoh#28] - -(24) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] - -(25) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] -Keys [4]: [i_product_name#29, i_brand#30, i_class#31, i_category#32] -Functions [1]: [avg(inv_quantity_on_hand#35)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#17] -Results [4]: [i_product_name#29, i_brand#30, i_class#31, avg(inv_quantity_on_hand#35)#17 AS qoh#36] - -(26) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#29, i_brand#30, i_class#31, qoh#36] -Keys [3]: [i_product_name#29, i_brand#30, i_class#31] -Functions [1]: [partial_avg(qoh#36)] -Aggregate Attributes [2]: [sum#37, count#38] -Results [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] - -(27) Exchange -Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] -Arguments: hashpartitioning(i_product_name#29, i_brand#30, i_class#31, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(28) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] -Keys [3]: [i_product_name#29, i_brand#30, i_class#31] -Functions [1]: [avg(qoh#36)] -Aggregate Attributes [1]: [avg(qoh#36)#41] -Results [5]: [i_product_name#29, i_brand#30, i_class#31, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] - -(29) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] - -(30) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] -Keys [4]: [i_product_name#44, i_brand#45, i_class#46, i_category#47] -Functions [1]: [avg(inv_quantity_on_hand#50)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#50)#17] -Results [3]: [i_product_name#44, i_brand#45, avg(inv_quantity_on_hand#50)#17 AS qoh#51] - -(31) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#44, i_brand#45, qoh#51] -Keys [2]: [i_product_name#44, i_brand#45] -Functions [1]: [partial_avg(qoh#51)] -Aggregate Attributes [2]: [sum#52, count#53] -Results [4]: [i_product_name#44, i_brand#45, sum#54, count#55] - -(32) Exchange -Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] -Arguments: hashpartitioning(i_product_name#44, i_brand#45, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(33) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] -Keys [2]: [i_product_name#44, i_brand#45] -Functions [1]: [avg(qoh#51)] -Aggregate Attributes [1]: [avg(qoh#51)#56] -Results [5]: [i_product_name#44, i_brand#45, null AS i_class#57, null AS i_category#58, avg(qoh#51)#56 AS qoh#59] - -(34) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] - -(35) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] -Keys [4]: [i_product_name#60, i_brand#61, i_class#62, i_category#63] -Functions [1]: [avg(inv_quantity_on_hand#66)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#66)#17] -Results [2]: [i_product_name#60, avg(inv_quantity_on_hand#66)#17 AS qoh#67] - -(36) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#60, qoh#67] -Keys [1]: [i_product_name#60] -Functions [1]: [partial_avg(qoh#67)] -Aggregate Attributes [2]: [sum#68, count#69] -Results [3]: [i_product_name#60, sum#70, count#71] - -(37) Exchange -Input [3]: [i_product_name#60, sum#70, count#71] -Arguments: hashpartitioning(i_product_name#60, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(38) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#60, sum#70, count#71] -Keys [1]: [i_product_name#60] -Functions [1]: [avg(qoh#67)] -Aggregate Attributes [1]: [avg(qoh#67)#72] -Results [5]: [i_product_name#60, null AS i_brand#73, null AS i_class#74, null AS i_category#75, avg(qoh#67)#72 AS qoh#76] - -(39) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] - -(40) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] -Keys [4]: [i_product_name#77, i_brand#78, i_class#79, i_category#80] -Functions [1]: [avg(inv_quantity_on_hand#83)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#83)#17] -Results [1]: [avg(inv_quantity_on_hand#83)#17 AS qoh#84] - -(41) HashAggregate [codegen id : 28] -Input [1]: [qoh#84] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#18] +Results [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, avg(inv_quantity_on_hand#3)#18 AS qoh#19] + +(23) HashAggregate [codegen id : 2] +Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#19] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] +Functions [1]: [partial_avg(qoh#19)] +Aggregate Attributes [2]: [sum#20, count#21] +Results [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#22, count#23] + +(24) HashAggregate [codegen id : 2] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#22, count#23] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] +Functions [1]: [avg(qoh#19)] +Aggregate Attributes [1]: [avg(qoh#19)#24] +Results [5]: [i_product_name#12 AS i_product_name#25, i_brand#9 AS i_brand#26, i_class#10 AS i_class#27, i_category#11 AS i_category#28, avg(qoh#19)#24 AS qoh#29] + +(25) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] + +(26) HashAggregate [codegen id : 4] +Input [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] +Keys [4]: [i_product_name#30, i_brand#31, i_class#32, i_category#33] +Functions [1]: [avg(inv_quantity_on_hand#36)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#36)#18] +Results [4]: [i_product_name#30, i_brand#31, i_class#32, avg(inv_quantity_on_hand#36)#18 AS qoh#37] + +(27) HashAggregate [codegen id : 4] +Input [4]: [i_product_name#30, i_brand#31, i_class#32, qoh#37] +Keys [3]: [i_product_name#30, i_brand#31, i_class#32] +Functions [1]: [partial_avg(qoh#37)] +Aggregate Attributes [2]: [sum#38, count#39] +Results [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] + +(28) Exchange +Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] +Arguments: hashpartitioning(i_product_name#30, i_brand#31, i_class#32, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(29) HashAggregate [codegen id : 5] +Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] +Keys [3]: [i_product_name#30, i_brand#31, i_class#32] +Functions [1]: [avg(qoh#37)] +Aggregate Attributes [1]: [avg(qoh#37)#42] +Results [5]: [i_product_name#30, i_brand#31, i_class#32, null AS i_category#43, avg(qoh#37)#42 AS qoh#44] + +(30) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#45, i_brand#46, i_class#47, i_category#48, sum#49, count#50] + +(31) HashAggregate [codegen id : 7] +Input [6]: [i_product_name#45, i_brand#46, i_class#47, i_category#48, sum#49, count#50] +Keys [4]: [i_product_name#45, i_brand#46, i_class#47, i_category#48] +Functions [1]: [avg(inv_quantity_on_hand#51)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#51)#18] +Results [3]: [i_product_name#45, i_brand#46, avg(inv_quantity_on_hand#51)#18 AS qoh#52] + +(32) HashAggregate [codegen id : 7] +Input [3]: [i_product_name#45, i_brand#46, qoh#52] +Keys [2]: [i_product_name#45, i_brand#46] +Functions [1]: [partial_avg(qoh#52)] +Aggregate Attributes [2]: [sum#53, count#54] +Results [4]: [i_product_name#45, i_brand#46, sum#55, count#56] + +(33) Exchange +Input [4]: [i_product_name#45, i_brand#46, sum#55, count#56] +Arguments: hashpartitioning(i_product_name#45, i_brand#46, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(34) HashAggregate [codegen id : 8] +Input [4]: [i_product_name#45, i_brand#46, sum#55, count#56] +Keys [2]: [i_product_name#45, i_brand#46] +Functions [1]: [avg(qoh#52)] +Aggregate Attributes [1]: [avg(qoh#52)#57] +Results [5]: [i_product_name#45, i_brand#46, null AS i_class#58, null AS i_category#59, avg(qoh#52)#57 AS qoh#60] + +(35) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, sum#65, count#66] + +(36) HashAggregate [codegen id : 10] +Input [6]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, sum#65, count#66] +Keys [4]: [i_product_name#61, i_brand#62, i_class#63, i_category#64] +Functions [1]: [avg(inv_quantity_on_hand#67)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#67)#18] +Results [2]: [i_product_name#61, avg(inv_quantity_on_hand#67)#18 AS qoh#68] + +(37) HashAggregate [codegen id : 10] +Input [2]: [i_product_name#61, qoh#68] +Keys [1]: [i_product_name#61] +Functions [1]: [partial_avg(qoh#68)] +Aggregate Attributes [2]: [sum#69, count#70] +Results [3]: [i_product_name#61, sum#71, count#72] + +(38) Exchange +Input [3]: [i_product_name#61, sum#71, count#72] +Arguments: hashpartitioning(i_product_name#61, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(39) HashAggregate [codegen id : 11] +Input [3]: [i_product_name#61, sum#71, count#72] +Keys [1]: [i_product_name#61] +Functions [1]: [avg(qoh#68)] +Aggregate Attributes [1]: [avg(qoh#68)#73] +Results [5]: [i_product_name#61, null AS i_brand#74, null AS i_class#75, null AS i_category#76, avg(qoh#68)#73 AS qoh#77] + +(40) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#78, i_brand#79, i_class#80, i_category#81, sum#82, count#83] + +(41) HashAggregate [codegen id : 13] +Input [6]: [i_product_name#78, i_brand#79, i_class#80, i_category#81, sum#82, count#83] +Keys [4]: [i_product_name#78, i_brand#79, i_class#80, i_category#81] +Functions [1]: [avg(inv_quantity_on_hand#84)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#84)#18] +Results [1]: [avg(inv_quantity_on_hand#84)#18 AS qoh#85] + +(42) HashAggregate [codegen id : 13] +Input [1]: [qoh#85] Keys: [] -Functions [1]: [partial_avg(qoh#84)] -Aggregate Attributes [2]: [sum#85, count#86] -Results [2]: [sum#87, count#88] +Functions [1]: [partial_avg(qoh#85)] +Aggregate Attributes [2]: [sum#86, count#87] +Results [2]: [sum#88, count#89] -(42) Exchange -Input [2]: [sum#87, count#88] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(43) Exchange +Input [2]: [sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(43) HashAggregate [codegen id : 29] -Input [2]: [sum#87, count#88] +(44) HashAggregate [codegen id : 14] +Input [2]: [sum#88, count#89] Keys: [] -Functions [1]: [avg(qoh#84)] -Aggregate Attributes [1]: [avg(qoh#84)#89] -Results [5]: [null AS i_product_name#90, null AS i_brand#91, null AS i_class#92, null AS i_category#93, avg(qoh#84)#89 AS qoh#94] +Functions [1]: [avg(qoh#85)] +Aggregate Attributes [1]: [avg(qoh#85)#90] +Results [5]: [null AS i_product_name#91, null AS i_brand#92, null AS i_class#93, null AS i_category#94, avg(qoh#85)#90 AS qoh#95] -(44) Union +(45) Union -(45) TakeOrderedAndProject -Input [5]: [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] -Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_class#26 ASC NULLS FIRST, i_category#27 ASC NULLS FIRST], [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] +(46) TakeOrderedAndProject +Input [5]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, qoh#29] +Arguments: 100, [qoh#29 ASC NULLS FIRST, i_product_name#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_class#27 ASC NULLS FIRST, i_category#28 ASC NULLS FIRST], [i_product_name#25, i_brand#26, i_class#27, i_category#28, qoh#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#95] +(47) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#95] -Condition : (((isnotnull(d_month_seq#95) AND (d_month_seq#95 >= 1212)) AND (d_month_seq#95 <= 1223)) AND isnotnull(d_date_sk#6)) +(48) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) -(48) CometProject -Input [2]: [d_date_sk#6, d_month_seq#95] +(49) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(49) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(50) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt index 3aa0745ad7..db81d1ba55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter Exchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -26,54 +26,48 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] - WholeStageCodegen (11) + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + WholeStageCodegen (5) HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] InputAdapter - Exchange [i_product_name,i_brand,i_class] #5 - WholeStageCodegen (10) + Exchange [i_product_name,i_brand,i_class] #6 + WholeStageCodegen (4) HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] InputAdapter - Exchange [i_product_name,i_brand] #6 - WholeStageCodegen (16) + Exchange [i_product_name,i_brand] #7 + WholeStageCodegen (7) HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (23) + WholeStageCodegen (11) HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] InputAdapter - Exchange [i_product_name] #7 - WholeStageCodegen (22) + Exchange [i_product_name] #8 + WholeStageCodegen (10) HashAggregate [i_product_name,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (29) + WholeStageCodegen (14) HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] InputAdapter - Exchange #8 - WholeStageCodegen (28) + Exchange #9 + WholeStageCodegen (13) HashAggregate [qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt index fc43e7271b..6c4d257c29 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt @@ -1,77 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (73) -+- Union (72) +TakeOrderedAndProject (71) ++- Union (70) :- * HashAggregate (28) : +- Exchange (27) : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) + : +- * ColumnarToRow (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (14) + : : : +- CometBroadcastHashJoin (13) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : : +- CometBroadcastExchange (12) + : : : +- CometProject (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (17) + : : +- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.store (15) + : +- CometBroadcastExchange (22) : +- CometFilter (21) : +- CometScan parquet spark_catalog.default.item (20) - :- * HashAggregate (50) - : +- Exchange (49) - : +- * HashAggregate (48) - : +- * Project (47) - : +- * BroadcastHashJoin Inner BuildRight (46) - : :- * Project (44) - : : +- * BroadcastHashJoin Inner BuildRight (43) - : : :- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * ColumnarToRow (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) - : : : : +- ReusedExchange (32) - : : : +- ReusedExchange (35) - : : +- BroadcastExchange (42) - : : +- * ColumnarToRow (41) - : : +- CometProject (40) - : : +- CometFilter (39) - : : +- CometScan parquet spark_catalog.default.store (38) - : +- ReusedExchange (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Project (56) - : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : :- * ColumnarToRow (53) - : : : : +- CometFilter (52) - : : : : +- CometScan parquet spark_catalog.default.store_sales (51) - : : : +- ReusedExchange (54) - : : +- ReusedExchange (57) - : +- ReusedExchange (60) - +- BroadcastExchange (66) - +- * ColumnarToRow (65) - +- CometFilter (64) - +- CometScan parquet spark_catalog.default.item (63) + :- * HashAggregate (49) + : +- Exchange (48) + : +- * HashAggregate (47) + : +- * ColumnarToRow (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometProject (42) + : : +- CometBroadcastHashJoin (41) + : : :- CometProject (36) + : : : +- CometBroadcastHashJoin (35) + : : : :- CometProject (33) + : : : : +- CometBroadcastHashJoin (32) + : : : : :- CometFilter (30) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) + : : : : +- ReusedExchange (31) + : : : +- ReusedExchange (34) + : : +- CometBroadcastExchange (40) + : : +- CometProject (39) + : : +- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.store (37) + : +- ReusedExchange (43) + +- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * ColumnarToRow (66) + +- CometProject (65) + +- CometBroadcastHashJoin (64) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (57) + : : +- CometBroadcastHashJoin (56) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometFilter (51) + : : : : +- CometScan parquet spark_catalog.default.store_sales (50) + : : : +- ReusedExchange (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (58) + +- CometBroadcastExchange (63) + +- CometFilter (62) + +- CometScan parquet spark_catalog.default.item (61) (1) Scan parquet spark_catalog.default.store_sales @@ -86,372 +84,361 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = F)) AND (cd_marital_status#12 = W)) AND (cd_education_status#13 = Primary )) AND isnotnull(cd_demo_sk#10)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) ColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange +(6) CometBroadcastExchange Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [cd_demo_sk#10] -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [cd_demo_sk#10] +Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +(8) CometProject Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] + +(9) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct + +(10) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) + +(11) CometProject +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14], [d_date_sk#14] -(11) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#14] +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +(14) CometProject Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(14) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] +(15) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((isnotnull(s_state#16) AND (s_state#16 = TN)) AND isnotnull(s_store_sk#15)) - -(16) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#16] +(16) CometFilter +Input [2]: [s_store_sk#16, s_state#17] +Condition : ((isnotnull(s_state#17) AND (s_state#17 = TN)) AND isnotnull(s_store_sk#16)) -(17) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_state#17] +Arguments: [s_store_sk#16, s_state#17] -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [2]: [s_store_sk#16, s_state#17] +Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight -(19) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#16] +(19) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#17] +Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] (20) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_item_id#18] +Output [2]: [i_item_sk#18, i_item_id#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (21) CometFilter -Input [2]: [i_item_sk#17, i_item_id#18] -Condition : isnotnull(i_item_sk#17) +Input [2]: [i_item_sk#18, i_item_id#19] +Condition : isnotnull(i_item_sk#18) -(22) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#17, i_item_id#18] +(22) CometBroadcastExchange +Input [2]: [i_item_sk#18, i_item_id#19] +Arguments: [i_item_sk#18, i_item_id#19] -(23) BroadcastExchange -Input [2]: [i_item_sk#17, i_item_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] +Right output [2]: [i_item_sk#18, i_item_id#19] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(24) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#19] +Arguments: [i_item_id#19, s_state#17, agg1#20, agg2#21, agg3#22, agg4#23], [i_item_id#19, s_state#17, ss_quantity#4 AS agg1#20, ss_list_price#5 AS agg2#21, ss_coupon_amt#7 AS agg3#22, ss_sales_price#6 AS agg4#23] -(25) Project [codegen id : 5] -Output [6]: [i_item_id#18, s_state#16, ss_quantity#4 AS agg1#19, ss_list_price#5 AS agg2#20, ss_coupon_amt#7 AS agg3#21, ss_sales_price#6 AS agg4#22] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16, i_item_sk#17, i_item_id#18] +(25) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_id#19, s_state#17, agg1#20, agg2#21, agg3#22, agg4#23] -(26) HashAggregate [codegen id : 5] -Input [6]: [i_item_id#18, s_state#16, agg1#19, agg2#20, agg3#21, agg4#22] -Keys [2]: [i_item_id#18, s_state#16] -Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), partial_avg(UnscaledValue(agg3#21)), partial_avg(UnscaledValue(agg4#22))] -Aggregate Attributes [8]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30] -Results [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] +(26) HashAggregate [codegen id : 1] +Input [6]: [i_item_id#19, s_state#17, agg1#20, agg2#21, agg3#22, agg4#23] +Keys [2]: [i_item_id#19, s_state#17] +Functions [4]: [partial_avg(agg1#20), partial_avg(UnscaledValue(agg2#21)), partial_avg(UnscaledValue(agg3#22)), partial_avg(UnscaledValue(agg4#23))] +Aggregate Attributes [8]: [sum#24, count#25, sum#26, count#27, sum#28, count#29, sum#30, count#31] +Results [10]: [i_item_id#19, s_state#17, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] (27) Exchange -Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] -Arguments: hashpartitioning(i_item_id#18, s_state#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [10]: [i_item_id#19, s_state#17, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] +Arguments: hashpartitioning(i_item_id#19, s_state#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 6] -Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] -Keys [2]: [i_item_id#18, s_state#16] -Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] -Aggregate Attributes [4]: [avg(agg1#19)#39, avg(UnscaledValue(agg2#20))#40, avg(UnscaledValue(agg3#21))#41, avg(UnscaledValue(agg4#22))#42] -Results [7]: [i_item_id#18, s_state#16, 0 AS g_state#43, avg(agg1#19)#39 AS agg1#44, cast((avg(UnscaledValue(agg2#20))#40 / 100.0) as decimal(11,6)) AS agg2#45, cast((avg(UnscaledValue(agg3#21))#41 / 100.0) as decimal(11,6)) AS agg3#46, cast((avg(UnscaledValue(agg4#22))#42 / 100.0) as decimal(11,6)) AS agg4#47] +(28) HashAggregate [codegen id : 2] +Input [10]: [i_item_id#19, s_state#17, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] +Keys [2]: [i_item_id#19, s_state#17] +Functions [4]: [avg(agg1#20), avg(UnscaledValue(agg2#21)), avg(UnscaledValue(agg3#22)), avg(UnscaledValue(agg4#23))] +Aggregate Attributes [4]: [avg(agg1#20)#40, avg(UnscaledValue(agg2#21))#41, avg(UnscaledValue(agg3#22))#42, avg(UnscaledValue(agg4#23))#43] +Results [7]: [i_item_id#19, s_state#17, 0 AS g_state#44, avg(agg1#20)#40 AS agg1#45, cast((avg(UnscaledValue(agg2#21))#41 / 100.0) as decimal(11,6)) AS agg2#46, cast((avg(UnscaledValue(agg3#22))#42 / 100.0) as decimal(11,6)) AS agg3#47, cast((avg(UnscaledValue(agg4#23))#43 / 100.0) as decimal(11,6)) AS agg4#48] (29) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#48, ss_cdemo_sk#49, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55] +Output [8]: [ss_item_sk#49, ss_cdemo_sk#50, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55, ss_sold_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#55), dynamicpruningexpression(ss_sold_date_sk#55 IN dynamicpruning#56)] +PartitionFilters: [isnotnull(ss_sold_date_sk#56), dynamicpruningexpression(ss_sold_date_sk#56 IN dynamicpruning#57)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (30) CometFilter -Input [8]: [ss_item_sk#48, ss_cdemo_sk#49, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55] -Condition : ((isnotnull(ss_cdemo_sk#49) AND isnotnull(ss_store_sk#50)) AND isnotnull(ss_item_sk#48)) +Input [8]: [ss_item_sk#49, ss_cdemo_sk#50, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_cdemo_sk#50) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_item_sk#49)) -(31) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#48, ss_cdemo_sk#49, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55] +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [cd_demo_sk#58] -(32) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#57] +(32) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#49, ss_cdemo_sk#50, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55, ss_sold_date_sk#56] +Right output [1]: [cd_demo_sk#58] +Arguments: [ss_cdemo_sk#50], [cd_demo_sk#58], Inner, BuildRight -(33) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#49] -Right keys [1]: [cd_demo_sk#57] -Join type: Inner -Join condition: None +(33) CometProject +Input [9]: [ss_item_sk#49, ss_cdemo_sk#50, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55, ss_sold_date_sk#56, cd_demo_sk#58] +Arguments: [ss_item_sk#49, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55, ss_sold_date_sk#56], [ss_item_sk#49, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55, ss_sold_date_sk#56] -(34) Project [codegen id : 11] -Output [7]: [ss_item_sk#48, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55] -Input [9]: [ss_item_sk#48, ss_cdemo_sk#49, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55, cd_demo_sk#57] +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [d_date_sk#59] -(35) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#58] +(35) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#49, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55, ss_sold_date_sk#56] +Right output [1]: [d_date_sk#59] +Arguments: [ss_sold_date_sk#56], [d_date_sk#59], Inner, BuildRight -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#55] -Right keys [1]: [d_date_sk#58] -Join type: Inner -Join condition: None +(36) CometProject +Input [8]: [ss_item_sk#49, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55, ss_sold_date_sk#56, d_date_sk#59] +Arguments: [ss_item_sk#49, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55], [ss_item_sk#49, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55] -(37) Project [codegen id : 11] -Output [6]: [ss_item_sk#48, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54] -Input [8]: [ss_item_sk#48, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55, d_date_sk#58] - -(38) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#59, s_state#60] +(37) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#60, s_state#61] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(39) CometFilter -Input [2]: [s_store_sk#59, s_state#60] -Condition : ((isnotnull(s_state#60) AND (s_state#60 = TN)) AND isnotnull(s_store_sk#59)) - -(40) CometProject -Input [2]: [s_store_sk#59, s_state#60] -Arguments: [s_store_sk#59], [s_store_sk#59] - -(41) ColumnarToRow [codegen id : 9] -Input [1]: [s_store_sk#59] - -(42) BroadcastExchange -Input [1]: [s_store_sk#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#50] -Right keys [1]: [s_store_sk#59] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 11] -Output [5]: [ss_item_sk#48, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54] -Input [7]: [ss_item_sk#48, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, s_store_sk#59] - -(45) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#61, i_item_id#62] - -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#48] -Right keys [1]: [i_item_sk#61] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 11] -Output [5]: [i_item_id#62, ss_quantity#51 AS agg1#63, ss_list_price#52 AS agg2#64, ss_coupon_amt#54 AS agg3#65, ss_sales_price#53 AS agg4#66] -Input [7]: [ss_item_sk#48, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, i_item_sk#61, i_item_id#62] - -(48) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#62, agg1#63, agg2#64, agg3#65, agg4#66] -Keys [1]: [i_item_id#62] -Functions [4]: [partial_avg(agg1#63), partial_avg(UnscaledValue(agg2#64)), partial_avg(UnscaledValue(agg3#65)), partial_avg(UnscaledValue(agg4#66))] -Aggregate Attributes [8]: [sum#67, count#68, sum#69, count#70, sum#71, count#72, sum#73, count#74] -Results [9]: [i_item_id#62, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] - -(49) Exchange -Input [9]: [i_item_id#62, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] -Arguments: hashpartitioning(i_item_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(50) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#62, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] -Keys [1]: [i_item_id#62] -Functions [4]: [avg(agg1#63), avg(UnscaledValue(agg2#64)), avg(UnscaledValue(agg3#65)), avg(UnscaledValue(agg4#66))] -Aggregate Attributes [4]: [avg(agg1#63)#83, avg(UnscaledValue(agg2#64))#84, avg(UnscaledValue(agg3#65))#85, avg(UnscaledValue(agg4#66))#86] -Results [7]: [i_item_id#62, null AS s_state#87, 1 AS g_state#88, avg(agg1#63)#83 AS agg1#89, cast((avg(UnscaledValue(agg2#64))#84 / 100.0) as decimal(11,6)) AS agg2#90, cast((avg(UnscaledValue(agg3#65))#85 / 100.0) as decimal(11,6)) AS agg3#91, cast((avg(UnscaledValue(agg4#66))#86 / 100.0) as decimal(11,6)) AS agg4#92] - -(51) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#93, ss_cdemo_sk#94, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100] +(38) CometFilter +Input [2]: [s_store_sk#60, s_state#61] +Condition : ((isnotnull(s_state#61) AND (s_state#61 = TN)) AND isnotnull(s_store_sk#60)) + +(39) CometProject +Input [2]: [s_store_sk#60, s_state#61] +Arguments: [s_store_sk#60], [s_store_sk#60] + +(40) CometBroadcastExchange +Input [1]: [s_store_sk#60] +Arguments: [s_store_sk#60] + +(41) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#49, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55] +Right output [1]: [s_store_sk#60] +Arguments: [ss_store_sk#51], [s_store_sk#60], Inner, BuildRight + +(42) CometProject +Input [7]: [ss_item_sk#49, ss_store_sk#51, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55, s_store_sk#60] +Arguments: [ss_item_sk#49, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55], [ss_item_sk#49, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55] + +(43) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#62, i_item_id#63] + +(44) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#49, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55] +Right output [2]: [i_item_sk#62, i_item_id#63] +Arguments: [ss_item_sk#49], [i_item_sk#62], Inner, BuildRight + +(45) CometProject +Input [7]: [ss_item_sk#49, ss_quantity#52, ss_list_price#53, ss_sales_price#54, ss_coupon_amt#55, i_item_sk#62, i_item_id#63] +Arguments: [i_item_id#63, agg1#64, agg2#65, agg3#66, agg4#67], [i_item_id#63, ss_quantity#52 AS agg1#64, ss_list_price#53 AS agg2#65, ss_coupon_amt#55 AS agg3#66, ss_sales_price#54 AS agg4#67] + +(46) ColumnarToRow [codegen id : 3] +Input [5]: [i_item_id#63, agg1#64, agg2#65, agg3#66, agg4#67] + +(47) HashAggregate [codegen id : 3] +Input [5]: [i_item_id#63, agg1#64, agg2#65, agg3#66, agg4#67] +Keys [1]: [i_item_id#63] +Functions [4]: [partial_avg(agg1#64), partial_avg(UnscaledValue(agg2#65)), partial_avg(UnscaledValue(agg3#66)), partial_avg(UnscaledValue(agg4#67))] +Aggregate Attributes [8]: [sum#68, count#69, sum#70, count#71, sum#72, count#73, sum#74, count#75] +Results [9]: [i_item_id#63, sum#76, count#77, sum#78, count#79, sum#80, count#81, sum#82, count#83] + +(48) Exchange +Input [9]: [i_item_id#63, sum#76, count#77, sum#78, count#79, sum#80, count#81, sum#82, count#83] +Arguments: hashpartitioning(i_item_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(49) HashAggregate [codegen id : 4] +Input [9]: [i_item_id#63, sum#76, count#77, sum#78, count#79, sum#80, count#81, sum#82, count#83] +Keys [1]: [i_item_id#63] +Functions [4]: [avg(agg1#64), avg(UnscaledValue(agg2#65)), avg(UnscaledValue(agg3#66)), avg(UnscaledValue(agg4#67))] +Aggregate Attributes [4]: [avg(agg1#64)#84, avg(UnscaledValue(agg2#65))#85, avg(UnscaledValue(agg3#66))#86, avg(UnscaledValue(agg4#67))#87] +Results [7]: [i_item_id#63, null AS s_state#88, 1 AS g_state#89, avg(agg1#64)#84 AS agg1#90, cast((avg(UnscaledValue(agg2#65))#85 / 100.0) as decimal(11,6)) AS agg2#91, cast((avg(UnscaledValue(agg3#66))#86 / 100.0) as decimal(11,6)) AS agg3#92, cast((avg(UnscaledValue(agg4#67))#87 / 100.0) as decimal(11,6)) AS agg4#93] + +(50) Scan parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ss_sold_date_sk#101), dynamicpruningexpression(ss_sold_date_sk#101 IN dynamicpruning#102)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(52) CometFilter -Input [8]: [ss_item_sk#93, ss_cdemo_sk#94, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100] -Condition : ((isnotnull(ss_cdemo_sk#94) AND isnotnull(ss_store_sk#95)) AND isnotnull(ss_item_sk#93)) - -(53) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#93, ss_cdemo_sk#94, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100] +(51) CometFilter +Input [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] +Condition : ((isnotnull(ss_cdemo_sk#95) AND isnotnull(ss_store_sk#96)) AND isnotnull(ss_item_sk#94)) -(54) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#102] +(52) ReusedExchange [Reuses operator id: 6] +Output [1]: [cd_demo_sk#103] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#94] -Right keys [1]: [cd_demo_sk#102] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] +Right output [1]: [cd_demo_sk#103] +Arguments: [ss_cdemo_sk#95], [cd_demo_sk#103], Inner, BuildRight -(56) Project [codegen id : 17] -Output [7]: [ss_item_sk#93, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100] -Input [9]: [ss_item_sk#93, ss_cdemo_sk#94, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100, cd_demo_sk#102] +(54) CometProject +Input [9]: [ss_item_sk#94, ss_cdemo_sk#95, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101, cd_demo_sk#103] +Arguments: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101], [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] -(57) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#103] +(55) ReusedExchange [Reuses operator id: 12] +Output [1]: [d_date_sk#104] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#100] -Right keys [1]: [d_date_sk#103] -Join type: Inner -Join condition: None +(56) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101] +Right output [1]: [d_date_sk#104] +Arguments: [ss_sold_date_sk#101], [d_date_sk#104], Inner, BuildRight -(59) Project [codegen id : 17] -Output [6]: [ss_item_sk#93, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99] -Input [8]: [ss_item_sk#93, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100, d_date_sk#103] +(57) CometProject +Input [8]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, ss_sold_date_sk#101, d_date_sk#104] +Arguments: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100], [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] -(60) ReusedExchange [Reuses operator id: 42] -Output [1]: [s_store_sk#104] +(58) ReusedExchange [Reuses operator id: 40] +Output [1]: [s_store_sk#105] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#95] -Right keys [1]: [s_store_sk#104] -Join type: Inner -Join condition: None +(59) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] +Right output [1]: [s_store_sk#105] +Arguments: [ss_store_sk#96], [s_store_sk#105], Inner, BuildRight -(62) Project [codegen id : 17] -Output [5]: [ss_item_sk#93, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99] -Input [7]: [ss_item_sk#93, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, s_store_sk#104] +(60) CometProject +Input [7]: [ss_item_sk#94, ss_store_sk#96, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, s_store_sk#105] +Arguments: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100], [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] -(63) Scan parquet spark_catalog.default.item -Output [1]: [i_item_sk#105] +(61) Scan parquet spark_catalog.default.item +Output [1]: [i_item_sk#106] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(64) CometFilter -Input [1]: [i_item_sk#105] -Condition : isnotnull(i_item_sk#105) +(62) CometFilter +Input [1]: [i_item_sk#106] +Condition : isnotnull(i_item_sk#106) -(65) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#105] +(63) CometBroadcastExchange +Input [1]: [i_item_sk#106] +Arguments: [i_item_sk#106] -(66) BroadcastExchange -Input [1]: [i_item_sk#105] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(64) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100] +Right output [1]: [i_item_sk#106] +Arguments: [ss_item_sk#94], [i_item_sk#106], Inner, BuildRight -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#93] -Right keys [1]: [i_item_sk#105] -Join type: Inner -Join condition: None +(65) CometProject +Input [6]: [ss_item_sk#94, ss_quantity#97, ss_list_price#98, ss_sales_price#99, ss_coupon_amt#100, i_item_sk#106] +Arguments: [agg1#107, agg2#108, agg3#109, agg4#110], [ss_quantity#97 AS agg1#107, ss_list_price#98 AS agg2#108, ss_coupon_amt#100 AS agg3#109, ss_sales_price#99 AS agg4#110] -(68) Project [codegen id : 17] -Output [4]: [ss_quantity#96 AS agg1#106, ss_list_price#97 AS agg2#107, ss_coupon_amt#99 AS agg3#108, ss_sales_price#98 AS agg4#109] -Input [6]: [ss_item_sk#93, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, i_item_sk#105] +(66) ColumnarToRow [codegen id : 5] +Input [4]: [agg1#107, agg2#108, agg3#109, agg4#110] -(69) HashAggregate [codegen id : 17] -Input [4]: [agg1#106, agg2#107, agg3#108, agg4#109] +(67) HashAggregate [codegen id : 5] +Input [4]: [agg1#107, agg2#108, agg3#109, agg4#110] Keys: [] -Functions [4]: [partial_avg(agg1#106), partial_avg(UnscaledValue(agg2#107)), partial_avg(UnscaledValue(agg3#108)), partial_avg(UnscaledValue(agg4#109))] -Aggregate Attributes [8]: [sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Results [8]: [sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125] +Functions [4]: [partial_avg(agg1#107), partial_avg(UnscaledValue(agg2#108)), partial_avg(UnscaledValue(agg3#109)), partial_avg(UnscaledValue(agg4#110))] +Aggregate Attributes [8]: [sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118] +Results [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] -(70) Exchange -Input [8]: [sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] +(68) Exchange +Input [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(71) HashAggregate [codegen id : 18] -Input [8]: [sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125] +(69) HashAggregate [codegen id : 6] +Input [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] Keys: [] -Functions [4]: [avg(agg1#106), avg(UnscaledValue(agg2#107)), avg(UnscaledValue(agg3#108)), avg(UnscaledValue(agg4#109))] -Aggregate Attributes [4]: [avg(agg1#106)#126, avg(UnscaledValue(agg2#107))#127, avg(UnscaledValue(agg3#108))#128, avg(UnscaledValue(agg4#109))#129] -Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#106)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#107))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#108))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#109))#129 / 100.0) as decimal(11,6)) AS agg4#136] +Functions [4]: [avg(agg1#107), avg(UnscaledValue(agg2#108)), avg(UnscaledValue(agg3#109)), avg(UnscaledValue(agg4#110))] +Aggregate Attributes [4]: [avg(agg1#107)#127, avg(UnscaledValue(agg2#108))#128, avg(UnscaledValue(agg3#109))#129, avg(UnscaledValue(agg4#110))#130] +Results [7]: [null AS i_item_id#131, null AS s_state#132, 1 AS g_state#133, avg(agg1#107)#127 AS agg1#134, cast((avg(UnscaledValue(agg2#108))#128 / 100.0) as decimal(11,6)) AS agg2#135, cast((avg(UnscaledValue(agg3#109))#129 / 100.0) as decimal(11,6)) AS agg3#136, cast((avg(UnscaledValue(agg4#110))#130 / 100.0) as decimal(11,6)) AS agg4#137] -(72) Union +(70) Union -(73) TakeOrderedAndProject -Input [7]: [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] -Arguments: 100, [i_item_id#18 ASC NULLS FIRST, s_state#16 ASC NULLS FIRST], [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] +(71) TakeOrderedAndProject +Input [7]: [i_item_id#19, s_state#17, g_state#44, agg1#45, agg2#46, agg3#47, agg4#48] +Arguments: 100, [i_item_id#19 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#19, s_state#17, g_state#44, agg1#45, agg2#46, agg3#47, agg4#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (78) -+- * ColumnarToRow (77) - +- CometProject (76) - +- CometFilter (75) - +- CometScan parquet spark_catalog.default.date_dim (74) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(74) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#137] +(72) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(75) CometFilter -Input [2]: [d_date_sk#14, d_year#137] -Condition : ((isnotnull(d_year#137) AND (d_year#137 = 1998)) AND isnotnull(d_date_sk#14)) +(73) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) -(76) CometProject -Input [2]: [d_date_sk#14, d_year#137] +(74) CometProject +Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(77) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(78) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#55 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#56 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#101 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt index 32f003798d..b09f5af824 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,s_state] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -26,92 +26,68 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (12) + CometBroadcastExchange #3 + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #5 + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter - Exchange [i_item_id] #6 - WholeStageCodegen (11) + Exchange [i_item_id] #7 + WholeStageCodegen (3) HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #8 + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [i_item_sk,i_item_id] #6 + WholeStageCodegen (6) HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter - Exchange #8 - WholeStageCodegen (17) + Exchange #9 + WholeStageCodegen (5) HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [s_store_sk] #7 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #8 + CometBroadcastExchange #10 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt index 387fa47f9b..92e11aabe8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt @@ -1,36 +1,37 @@ == Physical Plan == -* Sort (32) -+- Exchange (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (24) - : +- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (28) - +- * ColumnarToRow (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.customer (25) +* Sort (33) ++- Exchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Filter (25) + : +- * HashAggregate (24) + : +- Exchange (23) + : +- * ColumnarToRow (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.household_demographics (15) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.customer (26) (1) Scan parquet spark_catalog.default.store_sales @@ -45,174 +46,179 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#7] +(4) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(7) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +(8) CometProject Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#10, s_county#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] +(10) CometFilter +Input [2]: [s_store_sk#10, s_county#11] +Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] +(11) CometProject +Input [2]: [s_store_sk#10, s_county#11] +Arguments: [s_store_sk#10], [s_store_sk#10] -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#10] +Arguments: [s_store_sk#10] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +Right output [1]: [s_store_sk#10] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] +(14) CometProject +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -(14) Scan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +(15) Scan parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) +(16) CometFilter +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Condition : ((((isnotnull(hd_vehicle_count#15) AND ((hd_buy_potential#13 = >10000 ) OR (hd_buy_potential#13 = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN ((cast(hd_dep_count#14 as double) / cast(hd_vehicle_count#15 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#12)) -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] +(17) CometProject +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Arguments: [hd_demo_sk#12], [hd_demo_sk#12] -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#12] +Arguments: [hd_demo_sk#12] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(19) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Right output [1]: [hd_demo_sk#12] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None +(20) CometProject +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] +Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] +(21) CometHashAggregate Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) Exchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) ColumnarToRow [codegen id : 1] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] + +(23) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(23) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +(24) HashAggregate [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] +Aggregate Attributes [1]: [count(1)#17] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(24) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) +(25) Filter [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18] +Condition : ((cnt#18 >= 15) AND (cnt#18 <= 20)) -(25) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(26) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) +(27) CometFilter +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Condition : isnotnull(c_customer_sk#19) -(27) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(28) ColumnarToRow [codegen id : 2] +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(28) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(29) BroadcastExchange +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(29) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#19] Join type: Inner Join condition: None -(30) Project [codegen id : 6] -Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(31) Project [codegen id : 3] +Output [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18, c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(31) Exchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(32) Exchange +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: rangepartitioning(c_last_name#22 ASC NULLS FIRST, c_first_name#21 ASC NULLS FIRST, c_salutation#20 ASC NULLS FIRST, c_preferred_cust_flag#23 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 7] -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST], true, 0 +(33) Sort [codegen id : 4] +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: [c_last_name#22 ASC NULLS FIRST, c_first_name#21 ASC NULLS FIRST, c_salutation#20 ASC NULLS FIRST, c_preferred_cust_flag#23 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(33) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +(34) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_dom#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] -Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(35) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(35) CometProject -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +(36) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(37) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt index b473e48921..970562e5c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt @@ -1,24 +1,24 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] InputAdapter Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [cnt] HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter Exchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,27 +29,21 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange #5 + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange #6 + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt index ce8753277a..7900000853 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (28) + : : +- * Filter (27) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) + : : : :- * ColumnarToRow (12) + : : : : +- CometBroadcastHashJoin (11) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (10) + : : : : +- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) Scan parquet spark_catalog.default.customer @@ -54,228 +56,237 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#9] +(5) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(6) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(7) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] + +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] +(9) CometProject Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ss_customer_sk#6], [ss_customer_sk#6] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#6] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Right output [1]: [ss_customer_sk#6] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight + +(12) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(13) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#15] -(13) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#13] +(15) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +Right output [1]: [d_date_sk#15] +Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(16) CometProject +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] +Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +(17) ColumnarToRow [codegen id : 1] +Input [1]: [ws_bill_customer_sk#12] -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) BroadcastExchange +Input [1]: [ws_bill_customer_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 9] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] +Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#19] -(20) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#17] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(23) CometProject +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#14] -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] +(24) ColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#16] -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#14] +Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(27) Filter [codegen id : 5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(28) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(27) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +(29) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) +(30) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(29) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] +(31) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#20, ca_state#21] -(30) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(32) BroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(31) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(32) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#19] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#19] +(34) Project [codegen id : 5] +Output [2]: [c_current_cdemo_sk#4, ca_state#21] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(33) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(35) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(34) CometFilter -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Condition : isnotnull(cd_demo_sk#20) +(36) CometFilter +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Condition : isnotnull(cd_demo_sk#22) -(35) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(37) ColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(36) BroadcastExchange -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(38) BroadcastExchange +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#20] +Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 9] -Output [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Input [8]: [c_current_cdemo_sk#4, ca_state#19, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(40) Project [codegen id : 5] +Output [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(39) HashAggregate [codegen id : 9] -Input [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#23), partial_max(cd_dep_count#23), partial_sum(cd_dep_count#23), partial_avg(cd_dep_employed_count#24), partial_max(cd_dep_employed_count#24), partial_sum(cd_dep_employed_count#24), partial_avg(cd_dep_college_count#25), partial_max(cd_dep_college_count#25), partial_sum(cd_dep_college_count#25)] -Aggregate Attributes [13]: [count#26, sum#27, count#28, max#29, sum#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38] -Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] +(41) HashAggregate [codegen id : 5] +Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] +Aggregate Attributes [13]: [count#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40] +Results [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] -(40) Exchange -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(42) Exchange +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] +Arguments: hashpartitioning(ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(41) HashAggregate [codegen id : 10] -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] -Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] -Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] +(43) HashAggregate [codegen id : 6] +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] +Aggregate Attributes [10]: [count(1)#54, avg(cd_dep_count#25)#55, max(cd_dep_count#25)#56, sum(cd_dep_count#25)#57, avg(cd_dep_employed_count#26)#58, max(cd_dep_employed_count#26)#59, sum(cd_dep_employed_count#26)#60, avg(cd_dep_college_count#27)#61, max(cd_dep_college_count#27)#62, sum(cd_dep_college_count#27)#63] +Results [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#54 AS cnt1#64, avg(cd_dep_count#25)#55 AS avg(cd_dep_count)#65, max(cd_dep_count#25)#56 AS max(cd_dep_count)#66, sum(cd_dep_count#25)#57 AS sum(cd_dep_count)#67, cd_dep_employed_count#26, count(1)#54 AS cnt2#68, avg(cd_dep_employed_count#26)#58 AS avg(cd_dep_employed_count)#69, max(cd_dep_employed_count#26)#59 AS max(cd_dep_employed_count)#70, sum(cd_dep_employed_count#26)#60 AS sum(cd_dep_employed_count)#71, cd_dep_college_count#27, count(1)#54 AS cnt3#72, avg(cd_dep_college_count#27)#61 AS avg(cd_dep_college_count)#73, max(cd_dep_college_count#27)#62 AS max(cd_dep_college_count)#74, sum(cd_dep_college_count#27)#63 AS sum(cd_dep_college_count)#75] -(42) TakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] -Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] +(44) TakeOrderedAndProject +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#64, avg(cd_dep_count)#65, max(cd_dep_count)#66, sum(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, avg(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, sum(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, avg(cd_dep_college_count)#73, max(cd_dep_college_count)#74, sum(cd_dep_college_count)#75] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#64, avg(cd_dep_count)#65, max(cd_dep_count)#66, sum(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, avg(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, sum(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, avg(cd_dep_college_count)#73, max(cd_dep_college_count)#74, sum(cd_dep_college_count)#75] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(43) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#74, d_qoy#75] +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] -Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) +(46) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(45) CometProject -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +(47) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt index dc724ca919..a6b4add1fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (6) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,61 +13,57 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Filter [exists,exists] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt index 648b19933c..8eafc64ffb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt @@ -1,44 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.customer_demographics (31) +TakeOrderedAndProject (38) ++- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * ColumnarToRow (34) + +- CometProject (33) + +- CometBroadcastHashJoin (32) + :- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometBroadcastHashJoin (11) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : +- CometBroadcastExchange (10) + : : : +- CometProject (9) + : : : +- CometBroadcastHashJoin (8) + : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : +- CometBroadcastExchange (21) + : : +- CometUnion (20) + : : :- CometProject (15) + : : : +- CometBroadcastHashJoin (14) + : : : :- CometScan parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (13) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (17) + : +- CometBroadcastExchange (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer_address (24) + +- CometBroadcastExchange (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.customer_demographics (29) (1) Scan parquet spark_catalog.default.customer @@ -52,216 +50,212 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct + +(5) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) + +(6) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Arguments: [d_date_sk#7], [d_date_sk#7] -(6) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#7] +(7) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] +(9) CometProject Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#4] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ss_customer_sk#4] +Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(12) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#11] +(13) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customsk#12] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] +(15) CometProject +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +Arguments: [customsk#14], [ws_bill_customer_sk#10 AS customsk#14] (16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(17) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#18] -(18) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#16] +(18) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Right output [1]: [d_date_sk#18] +Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(19) CometProject +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] +Arguments: [customsk#19], [cs_ship_customer_sk#15 AS customsk#19] -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customsk#17] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] +(20) CometUnion +Child 0 Input [1]: [customsk#14] +Child 1 Input [1]: [customsk#19] -(21) Union +(21) CometBroadcastExchange +Input [1]: [customsk#14] +Arguments: [customsk#14] -(22) BroadcastExchange -Input [1]: [customsk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(22) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [customsk#14] +Arguments: [c_customer_sk#1], [customsk#14], LeftSemi, BuildRight -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#12] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +(23) CometProject Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(25) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +(24) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(26) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) - -(27) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] +(25) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(28) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(26) CometBroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: [ca_address_sk#20, ca_state#21] -(29) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [2]: [ca_address_sk#20, ca_state#21] +Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight -(30) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#19] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18, ca_state#19] +(28) CometProject +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20, ca_state#21] +Arguments: [c_current_cdemo_sk#2, ca_state#21], [c_current_cdemo_sk#2, ca_state#21] -(31) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(29) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(32) CometFilter -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Condition : isnotnull(cd_demo_sk#20) - -(33) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] - -(34) BroadcastExchange -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 9] -Output [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Input [8]: [c_current_cdemo_sk#2, ca_state#19, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] - -(37) HashAggregate [codegen id : 9] -Input [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#23), partial_max(cd_dep_count#23), partial_sum(cd_dep_count#23), partial_avg(cd_dep_employed_count#24), partial_max(cd_dep_employed_count#24), partial_sum(cd_dep_employed_count#24), partial_avg(cd_dep_college_count#25), partial_max(cd_dep_college_count#25), partial_sum(cd_dep_college_count#25)] -Aggregate Attributes [13]: [count#26, sum#27, count#28, max#29, sum#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38] -Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] - -(38) Exchange -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(39) HashAggregate [codegen id : 10] -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] -Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] -Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] - -(40) TakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] -Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] +(30) CometFilter +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Condition : isnotnull(cd_demo_sk#22) + +(31) CometBroadcastExchange +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] + +(32) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, ca_state#21] +Right output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight + +(33) CometProject +Input [8]: [c_current_cdemo_sk#2, ca_state#21, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27], [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] + +(34) ColumnarToRow [codegen id : 1] +Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] + +(35) HashAggregate [codegen id : 1] +Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] +Aggregate Attributes [13]: [count#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40] +Results [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] + +(36) Exchange +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] +Arguments: hashpartitioning(ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(37) HashAggregate [codegen id : 2] +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] +Aggregate Attributes [10]: [count(1)#54, avg(cd_dep_count#25)#55, max(cd_dep_count#25)#56, sum(cd_dep_count#25)#57, avg(cd_dep_employed_count#26)#58, max(cd_dep_employed_count#26)#59, sum(cd_dep_employed_count#26)#60, avg(cd_dep_college_count#27)#61, max(cd_dep_college_count#27)#62, sum(cd_dep_college_count#27)#63] +Results [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#54 AS cnt1#64, avg(cd_dep_count#25)#55 AS avg(cd_dep_count)#65, max(cd_dep_count#25)#56 AS max(cd_dep_count)#66, sum(cd_dep_count#25)#57 AS sum(cd_dep_count)#67, cd_dep_employed_count#26, count(1)#54 AS cnt2#68, avg(cd_dep_employed_count#26)#58 AS avg(cd_dep_employed_count)#69, max(cd_dep_employed_count#26)#59 AS max(cd_dep_employed_count)#70, sum(cd_dep_employed_count#26)#60 AS sum(cd_dep_employed_count)#71, cd_dep_college_count#27, count(1)#54 AS cnt3#72, avg(cd_dep_college_count#27)#61 AS avg(cd_dep_college_count)#73, max(cd_dep_college_count#27)#62 AS max(cd_dep_college_count)#74, sum(cd_dep_college_count#27)#63 AS sum(cd_dep_college_count)#75] + +(38) TakeOrderedAndProject +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#64, avg(cd_dep_count)#65, max(cd_dep_count)#66, sum(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, avg(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, sum(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, avg(cd_dep_college_count)#73, max(cd_dep_college_count)#74, sum(cd_dep_college_count)#75] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#64, avg(cd_dep_count)#65, max(cd_dep_count)#66, sum(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, avg(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, sum(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, avg(cd_dep_college_count)#73, max(cd_dep_college_count)#74, sum(cd_dep_college_count)#75] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(41) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#74, d_qoy#75] +(39) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] -Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 1999)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#7)) +(40) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) -(43) CometProject -Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] +(41) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(45) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt index e5cb940552..a95475f7b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt @@ -1,71 +1,52 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (2) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (1) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customsk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [c_current_cdemo_sk,ca_state] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #5 + CometUnion + CometProject [ws_bill_customer_sk] [customsk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometProject [cs_ship_customer_sk] [customsk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #6 + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange #7 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt index 2610a698cb..7b3ae5c675 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt @@ -1,45 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- Window (39) - +- * Sort (38) - +- Exchange (37) - +- * HashAggregate (36) - +- Exchange (35) - +- * HashAggregate (34) - +- Union (33) - :- * HashAggregate (22) - : +- Exchange (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.store (13) - :- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * HashAggregate (24) - : +- ReusedExchange (23) - +- * HashAggregate (32) - +- Exchange (31) - +- * HashAggregate (30) - +- * HashAggregate (29) - +- ReusedExchange (28) +TakeOrderedAndProject (42) ++- * Project (41) + +- Window (40) + +- * Sort (39) + +- Exchange (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- Union (34) + :- * HashAggregate (23) + : +- Exchange (22) + : +- * HashAggregate (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.item (9) + : +- CometBroadcastExchange (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.store (14) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * HashAggregate (25) + : +- ReusedExchange (24) + +- * HashAggregate (33) + +- Exchange (32) + +- * HashAggregate (31) + +- * HashAggregate (30) + +- ReusedExchange (29) (1) Scan parquet spark_catalog.default.store_sales @@ -54,226 +55,233 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#7] +(4) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#7, d_year#8] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight + +(8) CometProject Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -(7) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] +(9) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#9, i_class#10, i_category#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) - -(9) ColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#9, i_category#10] +(10) CometFilter +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Condition : isnotnull(i_item_sk#9) -(10) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Arguments: [i_item_sk#9, i_class#10, i_category#11] -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +Right output [3]: [i_item_sk#9, i_class#10, i_category#11] +Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight -(12) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#9, i_category#10] +(13) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#10, i_category#11] +Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_state#12] +(14) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#12, s_state#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [2]: [s_store_sk#11, s_state#12] -Condition : ((isnotnull(s_state#12) AND (s_state#12 = TN)) AND isnotnull(s_store_sk#11)) +(15) CometFilter +Input [2]: [s_store_sk#12, s_state#13] +Condition : ((isnotnull(s_state#13) AND (s_state#13 = TN)) AND isnotnull(s_store_sk#12)) -(15) CometProject -Input [2]: [s_store_sk#11, s_state#12] -Arguments: [s_store_sk#11], [s_store_sk#11] +(16) CometProject +Input [2]: [s_store_sk#12, s_state#13] +Arguments: [s_store_sk#12], [s_store_sk#12] -(16) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#11] +(17) CometBroadcastExchange +Input [1]: [s_store_sk#12] +Arguments: [s_store_sk#12] -(17) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) CometBroadcastHashJoin +Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] +Right output [1]: [s_store_sk#12] +Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(19) CometProject +Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11, s_store_sk#12] +Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11], [ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] -(19) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10, s_store_sk#11] +(20) ColumnarToRow [codegen id : 1] +Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] -(20) HashAggregate [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] -Keys [2]: [i_category#10, i_class#9] +(21) HashAggregate [codegen id : 1] +Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] +Keys [2]: [i_category#11, i_class#10] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#13, sum#14] -Results [4]: [i_category#10, i_class#9, sum#15, sum#16] +Aggregate Attributes [2]: [sum#14, sum#15] +Results [4]: [i_category#11, i_class#10, sum#16, sum#17] -(21) Exchange -Input [4]: [i_category#10, i_class#9, sum#15, sum#16] -Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) Exchange +Input [4]: [i_category#11, i_class#10, sum#16, sum#17] +Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] -Input [4]: [i_category#10, i_class#9, sum#15, sum#16] -Keys [2]: [i_category#10, i_class#9] +(23) HashAggregate [codegen id : 2] +Input [4]: [i_category#11, i_class#10, sum#16, sum#17] +Keys [2]: [i_category#11, i_class#10] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#17, sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#17,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2)) as decimal(38,11)) AS gross_margin#19, i_category#10, i_class#9, 0 AS t_category#20, 0 AS t_class#21, 0 AS lochierarchy#22] - -(23) ReusedExchange [Reuses operator id: 21] -Output [4]: [i_category#23, i_class#24, sum#25, sum#26] - -(24) HashAggregate [codegen id : 10] -Input [4]: [i_category#23, i_class#24, sum#25, sum#26] -Keys [2]: [i_category#23, i_class#24] -Functions [2]: [sum(UnscaledValue(ss_net_profit#27)), sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#27))#29, sum(UnscaledValue(ss_ext_sales_price#28))#30] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#27))#29,17,2) AS ss_net_profit#31, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#30,17,2) AS ss_ext_sales_price#32, i_category#23] - -(25) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#31, ss_ext_sales_price#32, i_category#23] -Keys [1]: [i_category#23] -Functions [2]: [partial_sum(ss_net_profit#31), partial_sum(ss_ext_sales_price#32)] -Aggregate Attributes [4]: [sum#33, isEmpty#34, sum#35, isEmpty#36] -Results [5]: [i_category#23, sum#37, isEmpty#38, sum#39, isEmpty#40] - -(26) Exchange -Input [5]: [i_category#23, sum#37, isEmpty#38, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(27) HashAggregate [codegen id : 11] -Input [5]: [i_category#23, sum#37, isEmpty#38, sum#39, isEmpty#40] -Keys [1]: [i_category#23] -Functions [2]: [sum(ss_net_profit#31), sum(ss_ext_sales_price#32)] -Aggregate Attributes [2]: [sum(ss_net_profit#31)#41, sum(ss_ext_sales_price#32)#42] -Results [6]: [(sum(ss_net_profit#31)#41 / sum(ss_ext_sales_price#32)#42) AS gross_margin#43, i_category#23, null AS i_class#44, 0 AS t_category#45, 1 AS t_class#46, 1 AS lochierarchy#47] - -(28) ReusedExchange [Reuses operator id: 21] -Output [4]: [i_category#48, i_class#49, sum#50, sum#51] - -(29) HashAggregate [codegen id : 16] -Input [4]: [i_category#48, i_class#49, sum#50, sum#51] -Keys [2]: [i_category#48, i_class#49] -Functions [2]: [sum(UnscaledValue(ss_net_profit#52)), sum(UnscaledValue(ss_ext_sales_price#53))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#52))#29, sum(UnscaledValue(ss_ext_sales_price#53))#30] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#29,17,2) AS ss_net_profit#54, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#53))#30,17,2) AS ss_ext_sales_price#55] - -(30) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#54, ss_ext_sales_price#55] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#18, sum(UnscaledValue(ss_ext_sales_price#3))#19] +Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#18,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#19,17,2)) as decimal(38,11)) AS gross_margin#20, i_category#11, i_class#10, 0 AS t_category#21, 0 AS t_class#22, 0 AS lochierarchy#23] + +(24) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#24, i_class#25, sum#26, sum#27] + +(25) HashAggregate [codegen id : 4] +Input [4]: [i_category#24, i_class#25, sum#26, sum#27] +Keys [2]: [i_category#24, i_class#25] +Functions [2]: [sum(UnscaledValue(ss_net_profit#28)), sum(UnscaledValue(ss_ext_sales_price#29))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#28))#30, sum(UnscaledValue(ss_ext_sales_price#29))#31] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#28))#30,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#29))#31,17,2) AS ss_ext_sales_price#33, i_category#24] + +(26) HashAggregate [codegen id : 4] +Input [3]: [ss_net_profit#32, ss_ext_sales_price#33, i_category#24] +Keys [1]: [i_category#24] +Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] +Aggregate Attributes [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] +Results [5]: [i_category#24, sum#38, isEmpty#39, sum#40, isEmpty#41] + +(27) Exchange +Input [5]: [i_category#24, sum#38, isEmpty#39, sum#40, isEmpty#41] +Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(28) HashAggregate [codegen id : 5] +Input [5]: [i_category#24, sum#38, isEmpty#39, sum#40, isEmpty#41] +Keys [1]: [i_category#24] +Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] +Aggregate Attributes [2]: [sum(ss_net_profit#32)#42, sum(ss_ext_sales_price#33)#43] +Results [6]: [(sum(ss_net_profit#32)#42 / sum(ss_ext_sales_price#33)#43) AS gross_margin#44, i_category#24, null AS i_class#45, 0 AS t_category#46, 1 AS t_class#47, 1 AS lochierarchy#48] + +(29) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#49, i_class#50, sum#51, sum#52] + +(30) HashAggregate [codegen id : 7] +Input [4]: [i_category#49, i_class#50, sum#51, sum#52] +Keys [2]: [i_category#49, i_class#50] +Functions [2]: [sum(UnscaledValue(ss_net_profit#53)), sum(UnscaledValue(ss_ext_sales_price#54))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#53))#30, sum(UnscaledValue(ss_ext_sales_price#54))#31] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#53))#30,17,2) AS ss_net_profit#55, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#54))#31,17,2) AS ss_ext_sales_price#56] + +(31) HashAggregate [codegen id : 7] +Input [2]: [ss_net_profit#55, ss_ext_sales_price#56] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#54), partial_sum(ss_ext_sales_price#55)] -Aggregate Attributes [4]: [sum#56, isEmpty#57, sum#58, isEmpty#59] -Results [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] +Functions [2]: [partial_sum(ss_net_profit#55), partial_sum(ss_ext_sales_price#56)] +Aggregate Attributes [4]: [sum#57, isEmpty#58, sum#59, isEmpty#60] +Results [4]: [sum#61, isEmpty#62, sum#63, isEmpty#64] -(31) Exchange -Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] +(32) Exchange +Input [4]: [sum#61, isEmpty#62, sum#63, isEmpty#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(32) HashAggregate [codegen id : 17] -Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] +(33) HashAggregate [codegen id : 8] +Input [4]: [sum#61, isEmpty#62, sum#63, isEmpty#64] Keys: [] -Functions [2]: [sum(ss_net_profit#54), sum(ss_ext_sales_price#55)] -Aggregate Attributes [2]: [sum(ss_net_profit#54)#64, sum(ss_ext_sales_price#55)#65] -Results [6]: [(sum(ss_net_profit#54)#64 / sum(ss_ext_sales_price#55)#65) AS gross_margin#66, null AS i_category#67, null AS i_class#68, 1 AS t_category#69, 1 AS t_class#70, 2 AS lochierarchy#71] +Functions [2]: [sum(ss_net_profit#55), sum(ss_ext_sales_price#56)] +Aggregate Attributes [2]: [sum(ss_net_profit#55)#65, sum(ss_ext_sales_price#56)#66] +Results [6]: [(sum(ss_net_profit#55)#65 / sum(ss_ext_sales_price#56)#66) AS gross_margin#67, null AS i_category#68, null AS i_class#69, 1 AS t_category#70, 1 AS t_class#71, 2 AS lochierarchy#72] -(33) Union +(34) Union -(34) HashAggregate [codegen id : 18] -Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +(35) HashAggregate [codegen id : 9] +Input [6]: [gross_margin#20, i_category#11, i_class#10, t_category#21, t_class#22, lochierarchy#23] +Keys [6]: [gross_margin#20, i_category#11, i_class#10, t_category#21, t_class#22, lochierarchy#23] Functions: [] Aggregate Attributes: [] -Results [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +Results [6]: [gross_margin#20, i_category#11, i_class#10, t_category#21, t_class#22, lochierarchy#23] -(35) Exchange -Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -Arguments: hashpartitioning(gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(36) Exchange +Input [6]: [gross_margin#20, i_category#11, i_class#10, t_category#21, t_class#22, lochierarchy#23] +Arguments: hashpartitioning(gross_margin#20, i_category#11, i_class#10, t_category#21, t_class#22, lochierarchy#23, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(36) HashAggregate [codegen id : 19] -Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +(37) HashAggregate [codegen id : 10] +Input [6]: [gross_margin#20, i_category#11, i_class#10, t_category#21, t_class#22, lochierarchy#23] +Keys [6]: [gross_margin#20, i_category#11, i_class#10, t_category#21, t_class#22, lochierarchy#23] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, CASE WHEN (t_class#21 = 0) THEN i_category#10 END AS _w0#72] +Results [5]: [gross_margin#20, i_category#11, i_class#10, lochierarchy#23, CASE WHEN (t_class#22 = 0) THEN i_category#11 END AS _w0#73] -(37) Exchange -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#72] -Arguments: hashpartitioning(lochierarchy#22, _w0#72, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(38) Exchange +Input [5]: [gross_margin#20, i_category#11, i_class#10, lochierarchy#23, _w0#73] +Arguments: hashpartitioning(lochierarchy#23, _w0#73, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) Sort [codegen id : 20] -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#72] -Arguments: [lochierarchy#22 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#19 ASC NULLS FIRST], false, 0 +(39) Sort [codegen id : 11] +Input [5]: [gross_margin#20, i_category#11, i_class#10, lochierarchy#23, _w0#73] +Arguments: [lochierarchy#23 ASC NULLS FIRST, _w0#73 ASC NULLS FIRST, gross_margin#20 ASC NULLS FIRST], false, 0 -(39) Window -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#72] -Arguments: [rank(gross_margin#19) windowspecdefinition(lochierarchy#22, _w0#72, gross_margin#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#22, _w0#72], [gross_margin#19 ASC NULLS FIRST] +(40) Window +Input [5]: [gross_margin#20, i_category#11, i_class#10, lochierarchy#23, _w0#73] +Arguments: [rank(gross_margin#20) windowspecdefinition(lochierarchy#23, _w0#73, gross_margin#20 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#74], [lochierarchy#23, _w0#73], [gross_margin#20 ASC NULLS FIRST] -(40) Project [codegen id : 21] -Output [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#73] -Input [6]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#72, rank_within_parent#73] +(41) Project [codegen id : 12] +Output [5]: [gross_margin#20, i_category#11, i_class#10, lochierarchy#23, rank_within_parent#74] +Input [6]: [gross_margin#20, i_category#11, i_class#10, lochierarchy#23, _w0#73, rank_within_parent#74] -(41) TakeOrderedAndProject -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#73] -Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#73] +(42) TakeOrderedAndProject +Input [5]: [gross_margin#20, i_category#11, i_class#10, lochierarchy#23, rank_within_parent#74] +Arguments: 100, [lochierarchy#23 DESC NULLS LAST, CASE WHEN (lochierarchy#23 = 0) THEN i_category#11 END ASC NULLS FIRST, rank_within_parent#74 ASC NULLS FIRST], [gross_margin#20, i_category#11, i_class#10, lochierarchy#23, rank_within_parent#74] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (46) -+- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.date_dim (42) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(42) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#74] +(43) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_year#8] 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 -(43) CometFilter -Input [2]: [d_date_sk#7, d_year#74] -Condition : ((isnotnull(d_year#74) AND (d_year#74 = 2001)) AND isnotnull(d_date_sk#7)) +(44) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(44) CometProject -Input [2]: [d_date_sk#7, d_year#74] +(45) CometProject +Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(45) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(46) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt index f265d20995..485fdc3e20 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt @@ -1,34 +1,34 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (21) + WholeStageCodegen (12) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (20) + WholeStageCodegen (11) Sort [lochierarchy,_w0,gross_margin] InputAdapter Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (19) + WholeStageCodegen (10) HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] [_w0] InputAdapter Exchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (18) + WholeStageCodegen (9) HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,t_category,t_class,lochierarchy,sum,sum] InputAdapter Exchange [i_category,i_class] #3 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -39,37 +39,31 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (11) + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange #7 + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [i_category] #7 - WholeStageCodegen (10) + Exchange [i_category] #8 + WholeStageCodegen (4) HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange #8 - WholeStageCodegen (16) + Exchange #9 + WholeStageCodegen (7) HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt index 9cb7b35cb4..81588ca21d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt @@ -14,22 +14,22 @@ TakeOrderedAndProject (45) : : +- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) + : : +- * ColumnarToRow (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.store (13) : +- BroadcastExchange (35) @@ -57,10 +57,7 @@ ReadSchema: struct Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] - -(4) Scan parquet spark_catalog.default.store_sales +(3) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -68,39 +65,46 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Condition : (isnotnull(ss_item_sk#4) AND isnotnull(ss_store_sk#5)) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#4] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Right output [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: [i_item_sk#1], [ss_item_sk#4], Inner, BuildRight -(9) Project [codegen id : 4] -Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +(7) CometProject Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7], [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_year#10, d_moy#11] + +(11) CometBroadcastHashJoin +Left output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(12) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] +(12) CometProject Input [8]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] +Arguments: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11], [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] (13) Scan parquet spark_catalog.default.store Output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] @@ -113,24 +117,23 @@ ReadSchema: struct Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] Condition : ((isnotnull(s_store_sk#12) AND isnotnull(s_store_name#13)) AND isnotnull(s_company_name#14)) -(15) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [s_store_sk#12, s_store_name#13, s_company_name#14] -(16) BroadcastExchange -Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] +Right output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [ss_store_sk#5], [s_store_sk#12], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] +(17) CometProject Input [9]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14], [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] + +(18) ColumnarToRow [codegen id : 1] +Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 1] Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#6))] @@ -139,9 +142,9 @@ Results [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_yea (20) Exchange Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 2] Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] @@ -150,9 +153,9 @@ Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_yea (22) Exchange Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(23) Sort [codegen id : 6] +(23) Sort [codegen id : 3] Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 @@ -160,7 +163,7 @@ Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_nam Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] +(25) Filter [codegen id : 4] Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) @@ -168,18 +171,18 @@ Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] -(27) Filter [codegen id : 22] +(27) Filter [codegen id : 13] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(28) Project [codegen id : 22] +(28) Project [codegen id : 13] Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] (29) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(30) HashAggregate [codegen id : 12] +(30) HashAggregate [codegen id : 6] Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] @@ -188,9 +191,9 @@ Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_y (31) Exchange Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 13] +(32) Sort [codegen id : 7] Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 @@ -198,28 +201,28 @@ Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_n Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(34) Project [codegen id : 14] +(34) Project [codegen id : 8] Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] (35) BroadcastExchange Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 22] +(36) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] +(37) Project [codegen id : 13] Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] (38) ReusedExchange [Reuses operator id: 31] Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -(39) Sort [codegen id : 20] +(39) Sort [codegen id : 11] Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 @@ -227,21 +230,21 @@ Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_n Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(41) Project [codegen id : 21] +(41) Project [codegen id : 12] Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] (42) BroadcastExchange Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 22] +(43) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] +(44) Project [codegen id : 13] Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] @@ -251,7 +254,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#11 ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (49) +- * ColumnarToRow (48) +- CometFilter (47) @@ -274,6 +277,6 @@ Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt index a548953052..a480db5ea8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,74 +8,67 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) + BroadcastExchange #7 + WholeStageCodegen (8) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) + WholeStageCodegen (7) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) + Exchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (6) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] InputAdapter ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #9 + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) + WholeStageCodegen (11) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt index 6591c8b8a2..01c4c5583a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt @@ -1,78 +1,81 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- Union (70) - :- * Project (23) - : +- * Filter (22) - : +- Window (21) - : +- * Sort (20) - : +- Window (19) - : +- * Sort (18) - : +- Exchange (17) - : +- * HashAggregate (16) - : +- Exchange (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.web_returns (5) - : +- ReusedExchange (11) - :- * Project (46) - : +- * Filter (45) - : +- Window (44) - : +- * Sort (43) - : +- Window (42) - : +- * Sort (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometScan parquet spark_catalog.default.catalog_returns (28) - : +- ReusedExchange (34) - +- * Project (69) - +- * Filter (68) - +- Window (67) - +- * Sort (66) - +- Window (65) - +- * Sort (64) - +- Exchange (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * ColumnarToRow (56) - : +- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometBroadcastExchange (50) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan parquet spark_catalog.default.store_sales (47) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.store_returns (51) - +- ReusedExchange (57) +TakeOrderedAndProject (77) ++- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * Sort (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- Exchange (18) + : +- * HashAggregate (17) + : +- * ColumnarToRow (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * Sort (44) + : +- Exchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * ColumnarToRow (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * Sort (67) + +- Exchange (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * ColumnarToRow (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) Scan parquet spark_catalog.default.web_sales @@ -113,344 +116,357 @@ Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_am (8) CometBroadcastHashJoin Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner +Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft (9) CometProject Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(10) ColumnarToRow [codegen id : 2] -Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +(10) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] +ReadSchema: struct + +(11) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(11) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#13] +(12) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Arguments: [d_date_sk#13], [d_date_sk#13] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: [d_date_sk#13] -(12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight -(13) Project [codegen id : 2] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] +(15) CometProject Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] + +(16) ColumnarToRow [codegen id : 1] +Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(14) HashAggregate [codegen id : 2] +(17) HashAggregate [codegen id : 1] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Aggregate Attributes [6]: [sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] +Results [7]: [ws_item_sk#1, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -(15) Exchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(18) Exchange +Input [7]: [ws_item_sk#1, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(16) HashAggregate [codegen id : 3] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(19) HashAggregate [codegen id : 2] +Input [7]: [ws_item_sk#1, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] +Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#28, sum(coalesce(ws_quantity#3, 0))#29, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#30, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#31] +Results [3]: [ws_item_sk#1 AS item#32, (cast(sum(coalesce(wr_return_quantity#10, 0))#28 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#29 as decimal(15,4))) AS return_ratio#33, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#30 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#31 as decimal(15,4))) AS currency_ratio#34] -(17) Exchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] +(20) Exchange +Input [3]: [item#32, return_ratio#33, currency_ratio#34] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(18) Sort [codegen id : 4] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 +(21) Sort [codegen id : 3] +Input [3]: [item#32, return_ratio#33, currency_ratio#34] +Arguments: [return_ratio#33 ASC NULLS FIRST], false, 0 -(19) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] +(22) Window +Input [3]: [item#32, return_ratio#33, currency_ratio#34] +Arguments: [rank(return_ratio#33) windowspecdefinition(return_ratio#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#35], [return_ratio#33 ASC NULLS FIRST] -(20) Sort [codegen id : 5] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 4] +Input [4]: [item#32, return_ratio#33, currency_ratio#34, return_rank#35] +Arguments: [currency_ratio#34 ASC NULLS FIRST], false, 0 -(21) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] +(24) Window +Input [4]: [item#32, return_ratio#33, currency_ratio#34, return_rank#35] +Arguments: [rank(currency_ratio#34) windowspecdefinition(currency_ratio#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#36], [currency_ratio#34 ASC NULLS FIRST] -(22) Filter [codegen id : 6] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) +(25) Filter [codegen id : 5] +Input [5]: [item#32, return_ratio#33, currency_ratio#34, return_rank#35, currency_rank#36] +Condition : ((return_rank#35 <= 10) OR (currency_rank#36 <= 10)) -(23) Project [codegen id : 6] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +(26) Project [codegen id : 5] +Output [5]: [web AS channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Input [5]: [item#32, return_ratio#33, currency_ratio#34, return_rank#35, currency_rank#36] -(24) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +(27) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_net_profit#42, cs_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(25) CometFilter -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) +(28) CometFilter +Input [6]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_net_profit#42, cs_sold_date_sk#43] +Condition : (((((((isnotnull(cs_net_profit#42) AND isnotnull(cs_net_paid#41)) AND isnotnull(cs_quantity#40)) AND (cs_net_profit#42 > 1.00)) AND (cs_net_paid#41 > 0.00)) AND (cs_quantity#40 > 0)) AND isnotnull(cs_order_number#39)) AND isnotnull(cs_item_sk#38)) -(26) CometProject -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(29) CometProject +Input [6]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_net_profit#42, cs_sold_date_sk#43] +Arguments: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43], [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43] -(27) CometBroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(30) CometBroadcastExchange +Input [5]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43] +Arguments: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43] -(28) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +(31) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48, cr_returned_date_sk#49] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(29) CometFilter -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) - -(30) CometProject -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(31) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner - -(32) CometProject -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] - -(33) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] - -(34) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#48] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] - -(37) HashAggregate [codegen id : 8] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] -Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] - -(38) Exchange -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(39) HashAggregate [codegen id : 9] -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] -Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] - -(40) Exchange -Input [3]: [item#65, return_ratio#66, currency_ratio#67] +(32) CometFilter +Input [5]: [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48, cr_returned_date_sk#49] +Condition : (((isnotnull(cr_return_amount#48) AND (cr_return_amount#48 > 10000.00)) AND isnotnull(cr_order_number#46)) AND isnotnull(cr_item_sk#45)) + +(33) CometProject +Input [5]: [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48, cr_returned_date_sk#49] +Arguments: [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48], [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48] + +(34) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43] +Right output [4]: [cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48] +Arguments: [cs_order_number#39, cs_item_sk#38], [cr_order_number#46, cr_item_sk#45], Inner, BuildLeft + +(35) CometProject +Input [9]: [cs_item_sk#38, cs_order_number#39, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_item_sk#45, cr_order_number#46, cr_return_quantity#47, cr_return_amount#48] +Arguments: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_return_quantity#47, cr_return_amount#48], [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_return_quantity#47, cr_return_amount#48] + +(36) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#50] + +(37) CometBroadcastHashJoin +Left output [6]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_return_quantity#47, cr_return_amount#48] +Right output [1]: [d_date_sk#50] +Arguments: [cs_sold_date_sk#43], [d_date_sk#50], Inner, BuildRight + +(38) CometProject +Input [7]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cs_sold_date_sk#43, cr_return_quantity#47, cr_return_amount#48, d_date_sk#50] +Arguments: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48], [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48] + +(39) ColumnarToRow [codegen id : 6] +Input [5]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48] + +(40) HashAggregate [codegen id : 6] +Input [5]: [cs_item_sk#38, cs_quantity#40, cs_net_paid#41, cr_return_quantity#47, cr_return_amount#48] +Keys [1]: [cs_item_sk#38] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#47, 0)), partial_sum(coalesce(cs_quantity#40, 0)), partial_sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#41 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#51, sum#52, sum#53, isEmpty#54, sum#55, isEmpty#56] +Results [7]: [cs_item_sk#38, sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] + +(41) Exchange +Input [7]: [cs_item_sk#38, sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Arguments: hashpartitioning(cs_item_sk#38, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(42) HashAggregate [codegen id : 7] +Input [7]: [cs_item_sk#38, sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Keys [1]: [cs_item_sk#38] +Functions [4]: [sum(coalesce(cr_return_quantity#47, 0)), sum(coalesce(cs_quantity#40, 0)), sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#41 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#47, 0))#63, sum(coalesce(cs_quantity#40, 0))#64, sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00))#65, sum(coalesce(cast(cs_net_paid#41 as decimal(12,2)), 0.00))#66] +Results [3]: [cs_item_sk#38 AS item#67, (cast(sum(coalesce(cr_return_quantity#47, 0))#63 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#40, 0))#64 as decimal(15,4))) AS return_ratio#68, (cast(sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00))#65 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#41 as decimal(12,2)), 0.00))#66 as decimal(15,4))) AS currency_ratio#69] + +(43) Exchange +Input [3]: [item#67, return_ratio#68, currency_ratio#69] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(41) Sort [codegen id : 10] -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 +(44) Sort [codegen id : 8] +Input [3]: [item#67, return_ratio#68, currency_ratio#69] +Arguments: [return_ratio#68 ASC NULLS FIRST], false, 0 -(42) Window -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] +(45) Window +Input [3]: [item#67, return_ratio#68, currency_ratio#69] +Arguments: [rank(return_ratio#68) windowspecdefinition(return_ratio#68 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#70], [return_ratio#68 ASC NULLS FIRST] -(43) Sort [codegen id : 11] -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 9] +Input [4]: [item#67, return_ratio#68, currency_ratio#69, return_rank#70] +Arguments: [currency_ratio#69 ASC NULLS FIRST], false, 0 -(44) Window -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] +(47) Window +Input [4]: [item#67, return_ratio#68, currency_ratio#69, return_rank#70] +Arguments: [rank(currency_ratio#69) windowspecdefinition(currency_ratio#69 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#71], [currency_ratio#69 ASC NULLS FIRST] -(45) Filter [codegen id : 12] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] -Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) +(48) Filter [codegen id : 10] +Input [5]: [item#67, return_ratio#68, currency_ratio#69, return_rank#70, currency_rank#71] +Condition : ((return_rank#70 <= 10) OR (currency_rank#71 <= 10)) -(46) Project [codegen id : 12] -Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +(49) Project [codegen id : 10] +Output [5]: [catalog AS channel#72, item#67, return_ratio#68, return_rank#70, currency_rank#71] +Input [5]: [item#67, return_ratio#68, currency_ratio#69, return_rank#70, currency_rank#71] -(47) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +(50) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_net_profit#77, ss_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(48) CometFilter -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) +(51) CometFilter +Input [6]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_net_profit#77, ss_sold_date_sk#78] +Condition : (((((((isnotnull(ss_net_profit#77) AND isnotnull(ss_net_paid#76)) AND isnotnull(ss_quantity#75)) AND (ss_net_profit#77 > 1.00)) AND (ss_net_paid#76 > 0.00)) AND (ss_quantity#75 > 0)) AND isnotnull(ss_ticket_number#74)) AND isnotnull(ss_item_sk#73)) -(49) CometProject -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(52) CometProject +Input [6]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_net_profit#77, ss_sold_date_sk#78] +Arguments: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78], [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78] -(50) CometBroadcastExchange -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(53) CometBroadcastExchange +Input [5]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78] +Arguments: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78] -(51) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +(54) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(52) CometFilter -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) - -(53) CometProject -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(54) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner - -(55) CometProject -Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] - -(56) ColumnarToRow [codegen id : 14] -Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] - -(57) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#83] - -(58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#76] -Right keys [1]: [d_date_sk#83] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 14] -Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] - -(60) HashAggregate [codegen id : 14] -Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Keys [1]: [ss_item_sk#71] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] -Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(61) Exchange -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(62) HashAggregate [codegen id : 15] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Keys [1]: [ss_item_sk#71] -Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] -Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] - -(63) Exchange -Input [3]: [item#100, return_ratio#101, currency_ratio#102] +(55) CometFilter +Input [5]: [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83, sr_returned_date_sk#84] +Condition : (((isnotnull(sr_return_amt#83) AND (sr_return_amt#83 > 10000.00)) AND isnotnull(sr_ticket_number#81)) AND isnotnull(sr_item_sk#80)) + +(56) CometProject +Input [5]: [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83, sr_returned_date_sk#84] +Arguments: [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83], [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83] + +(57) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78] +Right output [4]: [sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83] +Arguments: [ss_ticket_number#74, ss_item_sk#73], [sr_ticket_number#81, sr_item_sk#80], Inner, BuildLeft + +(58) CometProject +Input [9]: [ss_item_sk#73, ss_ticket_number#74, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_item_sk#80, sr_ticket_number#81, sr_return_quantity#82, sr_return_amt#83] +Arguments: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_return_quantity#82, sr_return_amt#83], [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_return_quantity#82, sr_return_amt#83] + +(59) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#85] + +(60) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_return_quantity#82, sr_return_amt#83] +Right output [1]: [d_date_sk#85] +Arguments: [ss_sold_date_sk#78], [d_date_sk#85], Inner, BuildRight + +(61) CometProject +Input [7]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, ss_sold_date_sk#78, sr_return_quantity#82, sr_return_amt#83, d_date_sk#85] +Arguments: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83], [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83] + +(62) ColumnarToRow [codegen id : 11] +Input [5]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83] + +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_item_sk#73, ss_quantity#75, ss_net_paid#76, sr_return_quantity#82, sr_return_amt#83] +Keys [1]: [ss_item_sk#73] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#82, 0)), partial_sum(coalesce(ss_quantity#75, 0)), partial_sum(coalesce(cast(sr_return_amt#83 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#76 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#86, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] +Results [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] + +(64) Exchange +Input [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Arguments: hashpartitioning(ss_item_sk#73, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(65) HashAggregate [codegen id : 12] +Input [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Keys [1]: [ss_item_sk#73] +Functions [4]: [sum(coalesce(sr_return_quantity#82, 0)), sum(coalesce(ss_quantity#75, 0)), sum(coalesce(cast(sr_return_amt#83 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#76 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#82, 0))#98, sum(coalesce(ss_quantity#75, 0))#99, sum(coalesce(cast(sr_return_amt#83 as decimal(12,2)), 0.00))#100, sum(coalesce(cast(ss_net_paid#76 as decimal(12,2)), 0.00))#101] +Results [3]: [ss_item_sk#73 AS item#102, (cast(sum(coalesce(sr_return_quantity#82, 0))#98 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#75, 0))#99 as decimal(15,4))) AS return_ratio#103, (cast(sum(coalesce(cast(sr_return_amt#83 as decimal(12,2)), 0.00))#100 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#76 as decimal(12,2)), 0.00))#101 as decimal(15,4))) AS currency_ratio#104] + +(66) Exchange +Input [3]: [item#102, return_ratio#103, currency_ratio#104] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(64) Sort [codegen id : 16] -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 +(67) Sort [codegen id : 13] +Input [3]: [item#102, return_ratio#103, currency_ratio#104] +Arguments: [return_ratio#103 ASC NULLS FIRST], false, 0 -(65) Window -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] +(68) Window +Input [3]: [item#102, return_ratio#103, currency_ratio#104] +Arguments: [rank(return_ratio#103) windowspecdefinition(return_ratio#103 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#105], [return_ratio#103 ASC NULLS FIRST] -(66) Sort [codegen id : 17] -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 +(69) Sort [codegen id : 14] +Input [4]: [item#102, return_ratio#103, currency_ratio#104, return_rank#105] +Arguments: [currency_ratio#104 ASC NULLS FIRST], false, 0 -(67) Window -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] +(70) Window +Input [4]: [item#102, return_ratio#103, currency_ratio#104, return_rank#105] +Arguments: [rank(currency_ratio#104) windowspecdefinition(currency_ratio#104 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#106], [currency_ratio#104 ASC NULLS FIRST] -(68) Filter [codegen id : 18] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) +(71) Filter [codegen id : 15] +Input [5]: [item#102, return_ratio#103, currency_ratio#104, return_rank#105, currency_rank#106] +Condition : ((return_rank#105 <= 10) OR (currency_rank#106 <= 10)) -(69) Project [codegen id : 18] -Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] +(72) Project [codegen id : 15] +Output [5]: [store AS channel#107, item#102, return_ratio#103, return_rank#105, currency_rank#106] +Input [5]: [item#102, return_ratio#103, currency_ratio#104, return_rank#105, currency_rank#106] -(70) Union +(73) Union -(71) HashAggregate [codegen id : 19] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(74) HashAggregate [codegen id : 16] +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Keys [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] -(72) Exchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(75) Exchange +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Arguments: hashpartitioning(channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(73) HashAggregate [codegen id : 20] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(76) HashAggregate [codegen id : 17] +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Keys [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] -(74) TakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(77) TakeOrderedAndProject +Input [5]: [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] +Arguments: 100, [channel#37 ASC NULLS FIRST, return_rank#35 ASC NULLS FIRST, currency_rank#36 ASC NULLS FIRST, item#32 ASC NULLS FIRST], [channel#37, item#32, return_ratio#33, return_rank#35, currency_rank#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan parquet spark_catalog.default.date_dim (75) +BroadcastExchange (82) ++- * ColumnarToRow (81) + +- CometProject (80) + +- CometFilter (79) + +- CometScan parquet spark_catalog.default.date_dim (78) -(75) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#106, d_moy#107] +(78) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] -Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) +(79) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(77) CometProject -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +(80) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(78) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(79) BroadcastExchange +(82) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt index 43ebf34cc0..d1480ef10b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt @@ -1,35 +1,35 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (20) + WholeStageCodegen (17) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (19) + WholeStageCodegen (16) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (5) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (5) + WholeStageCodegen (4) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (4) + WholeStageCodegen (3) Sort [return_ratio] InputAdapter Exchange #2 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ws_item_sk] #3 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] CometBroadcastExchange #4 @@ -47,34 +47,36 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_return_amt,wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (12) + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (11) + WholeStageCodegen (9) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (10) + WholeStageCodegen (8) Sort [return_ratio] InputAdapter - Exchange #6 - WholeStageCodegen (9) + Exchange #7 + WholeStageCodegen (7) HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [cs_item_sk] #7 - WholeStageCodegen (8) + Exchange [cs_item_sk] #8 + WholeStageCodegen (6) HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #8 + CometBroadcastExchange #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] @@ -82,34 +84,33 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_return_amount,cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (18) + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (15) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (17) + WholeStageCodegen (14) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (16) + WholeStageCodegen (13) Sort [return_ratio] InputAdapter - Exchange #9 - WholeStageCodegen (15) + Exchange #10 + WholeStageCodegen (12) HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (14) + Exchange [ss_item_sk] #11 + WholeStageCodegen (11) HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #11 + CometBroadcastExchange #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] @@ -117,5 +118,4 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt index 3892f250a7..a5ad8c2930 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt @@ -1,71 +1,74 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Filter (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- Window (56) - : +- * Sort (55) - : +- Exchange (54) - : +- * Project (53) - : +- * Filter (52) - : +- * SortMergeJoin FullOuter (51) - : :- * Sort (25) - : : +- Exchange (24) - : : +- * HashAggregate (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (13) - : : : +- Window (12) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (18) - : : +- * Project (17) - : : +- Window (16) - : : +- * Sort (15) - : : +- ReusedExchange (14) - : +- * Sort (50) - : +- Exchange (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (38) - : : +- Window (37) - : : +- * Sort (36) - : : +- Exchange (35) - : : +- * HashAggregate (34) - : : +- Exchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- CometFilter (27) - : : : +- CometScan parquet spark_catalog.default.store_sales (26) - : : +- ReusedExchange (29) - : +- BroadcastExchange (43) - : +- * Project (42) - : +- Window (41) - : +- * Sort (40) - : +- ReusedExchange (39) - +- BroadcastExchange (61) - +- * Project (60) - +- Window (59) - +- * Sort (58) - +- ReusedExchange (57) +TakeOrderedAndProject (70) ++- * Filter (69) + +- * HashAggregate (68) + +- * HashAggregate (67) + +- * Project (66) + +- * BroadcastHashJoin Inner BuildRight (65) + :- Window (59) + : +- * Sort (58) + : +- Exchange (57) + : +- * Project (56) + : +- * Filter (55) + : +- * SortMergeJoin FullOuter (54) + : :- * Sort (28) + : : +- Exchange (27) + : : +- * HashAggregate (26) + : : +- Exchange (25) + : : +- * HashAggregate (24) + : : +- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Project (16) + : : : +- Window (15) + : : : +- * Sort (14) + : : : +- Exchange (13) + : : : +- * HashAggregate (12) + : : : +- Exchange (11) + : : : +- * HashAggregate (10) + : : : +- * ColumnarToRow (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (21) + : : +- * Project (20) + : : +- Window (19) + : : +- * Sort (18) + : : +- ReusedExchange (17) + : +- * Sort (53) + : +- Exchange (52) + : +- * HashAggregate (51) + : +- Exchange (50) + : +- * HashAggregate (49) + : +- * Project (48) + : +- * BroadcastHashJoin Inner BuildRight (47) + : :- * Project (41) + : : +- Window (40) + : : +- * Sort (39) + : : +- Exchange (38) + : : +- * HashAggregate (37) + : : +- Exchange (36) + : : +- * HashAggregate (35) + : : +- * ColumnarToRow (34) + : : +- CometProject (33) + : : +- CometBroadcastHashJoin (32) + : : :- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.store_sales (29) + : : +- ReusedExchange (31) + : +- BroadcastExchange (46) + : +- * Project (45) + : +- Window (44) + : +- * Sort (43) + : +- ReusedExchange (42) + +- BroadcastExchange (64) + +- * Project (63) + +- Window (62) + +- * Sort (61) + +- ReusedExchange (60) (1) Scan parquet spark_catalog.default.web_sales @@ -80,337 +83,351 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 72] -Output [2]: [d_date_sk#5, d_date#6] +(4) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] +(6) CometBroadcastExchange +Input [2]: [d_date_sk#5, d_date#6] +Arguments: [d_date_sk#5, d_date#6] + +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +Right output [2]: [d_date_sk#5, d_date#6] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] +Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] -(7) HashAggregate [codegen id : 2] +(9) ColumnarToRow [codegen id : 1] +Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] + +(10) HashAggregate [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] +Aggregate Attributes [1]: [sum#8] +Results [3]: [ws_item_sk#1, d_date#6, sum#9] -(8) Exchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] +(11) Exchange +Input [3]: [ws_item_sk#1, d_date#6, sum#9] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) HashAggregate [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] +(12) HashAggregate [codegen id : 2] +Input [3]: [ws_item_sk#1, d_date#6, sum#9] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#10] +Results [4]: [ws_item_sk#1 AS item_sk#11, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#10,17,2) AS sumws#12, ws_item_sk#1] -(10) Exchange -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +(13) Exchange +Input [4]: [item_sk#11, d_date#6, sumws#12, ws_item_sk#1] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) Sort [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +(14) Sort [codegen id : 3] +Input [4]: [item_sk#11, d_date#6, sumws#12, ws_item_sk#1] Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(12) Window -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(15) Window +Input [4]: [item_sk#11, d_date#6, sumws#12, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#13], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(13) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] -Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] +(16) Project [codegen id : 8] +Output [4]: [item_sk#11, d_date#6, sumws#12, rk#13] +Input [5]: [item_sk#11, d_date#6, sumws#12, ws_item_sk#1, rk#13] -(14) ReusedExchange [Reuses operator id: 10] -Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] +(17) ReusedExchange [Reuses operator id: 13] +Output [4]: [item_sk#14, d_date#15, sumws#16, ws_item_sk#17] -(15) Sort [codegen id : 8] -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST], false, 0 +(18) Sort [codegen id : 6] +Input [4]: [item_sk#14, d_date#15, sumws#16, ws_item_sk#17] +Arguments: [ws_item_sk#17 ASC NULLS FIRST, d_date#15 ASC NULLS FIRST], false, 0 -(16) Window -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] +(19) Window +Input [4]: [item_sk#14, d_date#15, sumws#16, ws_item_sk#17] +Arguments: [row_number() windowspecdefinition(ws_item_sk#17, d_date#15 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#18], [ws_item_sk#17], [d_date#15 ASC NULLS FIRST] -(17) Project [codegen id : 9] -Output [3]: [item_sk#13, sumws#15, rk#17] -Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] +(20) Project [codegen id : 7] +Output [3]: [item_sk#14, sumws#16, rk#18] +Input [5]: [item_sk#14, d_date#15, sumws#16, ws_item_sk#17, rk#18] -(18) BroadcastExchange -Input [3]: [item_sk#13, sumws#15, rk#17] +(21) BroadcastExchange +Input [3]: [item_sk#14, sumws#16, rk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#13] +(22) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [item_sk#11] +Right keys [1]: [item_sk#14] Join type: Inner -Join condition: (rk#12 >= rk#17) - -(20) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] - -(21) HashAggregate [codegen id : 10] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#15)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(22) Exchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(23) HashAggregate [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [sum(sumws#15)] -Aggregate Attributes [1]: [sum(sumws#15)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#15)#22 AS cume_sales#23] - -(24) Exchange -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(25) Sort [codegen id : 12] -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 - -(26) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Join condition: (rk#13 >= rk#18) + +(23) Project [codegen id : 8] +Output [4]: [item_sk#11, d_date#6, sumws#12, sumws#16] +Input [7]: [item_sk#11, d_date#6, sumws#12, rk#13, item_sk#14, sumws#16, rk#18] + +(24) HashAggregate [codegen id : 8] +Input [4]: [item_sk#11, d_date#6, sumws#12, sumws#16] +Keys [3]: [item_sk#11, d_date#6, sumws#12] +Functions [1]: [partial_sum(sumws#16)] +Aggregate Attributes [2]: [sum#19, isEmpty#20] +Results [5]: [item_sk#11, d_date#6, sumws#12, sum#21, isEmpty#22] + +(25) Exchange +Input [5]: [item_sk#11, d_date#6, sumws#12, sum#21, isEmpty#22] +Arguments: hashpartitioning(item_sk#11, d_date#6, sumws#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(26) HashAggregate [codegen id : 9] +Input [5]: [item_sk#11, d_date#6, sumws#12, sum#21, isEmpty#22] +Keys [3]: [item_sk#11, d_date#6, sumws#12] +Functions [1]: [sum(sumws#16)] +Aggregate Attributes [1]: [sum(sumws#16)#23] +Results [3]: [item_sk#11, d_date#6, sum(sumws#16)#23 AS cume_sales#24] + +(27) Exchange +Input [3]: [item_sk#11, d_date#6, cume_sales#24] +Arguments: hashpartitioning(item_sk#11, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(28) Sort [codegen id : 10] +Input [3]: [item_sk#11, d_date#6, cume_sales#24] +Arguments: [item_sk#11 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 + +(29) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#25, ss_sales_price#26, ss_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(27) CometFilter -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) +(30) CometFilter +Input [3]: [ss_item_sk#25, ss_sales_price#26, ss_sold_date_sk#27] +Condition : isnotnull(ss_item_sk#25) -(28) ColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +(31) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#29, d_date#30] -(29) ReusedExchange [Reuses operator id: 72] -Output [2]: [d_date_sk#28, d_date#29] +(32) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#25, ss_sales_price#26, ss_sold_date_sk#27] +Right output [2]: [d_date_sk#29, d_date#30] +Arguments: [ss_sold_date_sk#27], [d_date_sk#29], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#28] -Join type: Inner -Join condition: None +(33) CometProject +Input [5]: [ss_item_sk#25, ss_sales_price#26, ss_sold_date_sk#27, d_date_sk#29, d_date#30] +Arguments: [ss_item_sk#25, ss_sales_price#26, d_date#30], [ss_item_sk#25, ss_sales_price#26, d_date#30] -(31) Project [codegen id : 14] -Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] -Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] - -(32) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#30] -Results [3]: [ss_item_sk#24, d_date#29, sum#31] - -(33) Exchange -Input [3]: [ss_item_sk#24, d_date#29, sum#31] -Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(34) HashAggregate [codegen id : 15] -Input [3]: [ss_item_sk#24, d_date#29, sum#31] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#32] -Results [4]: [ss_item_sk#24 AS item_sk#33, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#32,17,2) AS sumss#34, ss_item_sk#24] - -(35) Exchange -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(36) Sort [codegen id : 16] -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 - -(37) Window -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] - -(38) Project [codegen id : 22] -Output [4]: [item_sk#33, d_date#29, sumss#34, rk#35] -Input [5]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24, rk#35] - -(39) ReusedExchange [Reuses operator id: 35] -Output [4]: [item_sk#36, d_date#37, sumss#38, ss_item_sk#39] - -(40) Sort [codegen id : 20] -Input [4]: [item_sk#36, d_date#37, sumss#38, ss_item_sk#39] -Arguments: [ss_item_sk#39 ASC NULLS FIRST, d_date#37 ASC NULLS FIRST], false, 0 - -(41) Window -Input [4]: [item_sk#36, d_date#37, sumss#38, ss_item_sk#39] -Arguments: [row_number() windowspecdefinition(ss_item_sk#39, d_date#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#40], [ss_item_sk#39], [d_date#37 ASC NULLS FIRST] - -(42) Project [codegen id : 21] -Output [3]: [item_sk#36, sumss#38, rk#40] -Input [5]: [item_sk#36, d_date#37, sumss#38, ss_item_sk#39, rk#40] - -(43) BroadcastExchange -Input [3]: [item_sk#36, sumss#38, rk#40] +(34) ColumnarToRow [codegen id : 11] +Input [3]: [ss_item_sk#25, ss_sales_price#26, d_date#30] + +(35) HashAggregate [codegen id : 11] +Input [3]: [ss_item_sk#25, ss_sales_price#26, d_date#30] +Keys [2]: [ss_item_sk#25, d_date#30] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#26))] +Aggregate Attributes [1]: [sum#31] +Results [3]: [ss_item_sk#25, d_date#30, sum#32] + +(36) Exchange +Input [3]: [ss_item_sk#25, d_date#30, sum#32] +Arguments: hashpartitioning(ss_item_sk#25, d_date#30, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(37) HashAggregate [codegen id : 12] +Input [3]: [ss_item_sk#25, d_date#30, sum#32] +Keys [2]: [ss_item_sk#25, d_date#30] +Functions [1]: [sum(UnscaledValue(ss_sales_price#26))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#26))#33] +Results [4]: [ss_item_sk#25 AS item_sk#34, d_date#30, MakeDecimal(sum(UnscaledValue(ss_sales_price#26))#33,17,2) AS sumss#35, ss_item_sk#25] + +(38) Exchange +Input [4]: [item_sk#34, d_date#30, sumss#35, ss_item_sk#25] +Arguments: hashpartitioning(ss_item_sk#25, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(39) Sort [codegen id : 13] +Input [4]: [item_sk#34, d_date#30, sumss#35, ss_item_sk#25] +Arguments: [ss_item_sk#25 ASC NULLS FIRST, d_date#30 ASC NULLS FIRST], false, 0 + +(40) Window +Input [4]: [item_sk#34, d_date#30, sumss#35, ss_item_sk#25] +Arguments: [row_number() windowspecdefinition(ss_item_sk#25, d_date#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [ss_item_sk#25], [d_date#30 ASC NULLS FIRST] + +(41) Project [codegen id : 18] +Output [4]: [item_sk#34, d_date#30, sumss#35, rk#36] +Input [5]: [item_sk#34, d_date#30, sumss#35, ss_item_sk#25, rk#36] + +(42) ReusedExchange [Reuses operator id: 38] +Output [4]: [item_sk#37, d_date#38, sumss#39, ss_item_sk#40] + +(43) Sort [codegen id : 16] +Input [4]: [item_sk#37, d_date#38, sumss#39, ss_item_sk#40] +Arguments: [ss_item_sk#40 ASC NULLS FIRST, d_date#38 ASC NULLS FIRST], false, 0 + +(44) Window +Input [4]: [item_sk#37, d_date#38, sumss#39, ss_item_sk#40] +Arguments: [row_number() windowspecdefinition(ss_item_sk#40, d_date#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#41], [ss_item_sk#40], [d_date#38 ASC NULLS FIRST] + +(45) Project [codegen id : 17] +Output [3]: [item_sk#37, sumss#39, rk#41] +Input [5]: [item_sk#37, d_date#38, sumss#39, ss_item_sk#40, rk#41] + +(46) BroadcastExchange +Input [3]: [item_sk#37, sumss#39, rk#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(44) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [item_sk#33] -Right keys [1]: [item_sk#36] +(47) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_sk#34] +Right keys [1]: [item_sk#37] Join type: Inner -Join condition: (rk#35 >= rk#40) - -(45) Project [codegen id : 22] -Output [4]: [item_sk#33, d_date#29, sumss#34, sumss#38] -Input [7]: [item_sk#33, d_date#29, sumss#34, rk#35, item_sk#36, sumss#38, rk#40] - -(46) HashAggregate [codegen id : 22] -Input [4]: [item_sk#33, d_date#29, sumss#34, sumss#38] -Keys [3]: [item_sk#33, d_date#29, sumss#34] -Functions [1]: [partial_sum(sumss#38)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] - -(47) Exchange -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -Arguments: hashpartitioning(item_sk#33, d_date#29, sumss#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(48) HashAggregate [codegen id : 23] -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -Keys [3]: [item_sk#33, d_date#29, sumss#34] -Functions [1]: [sum(sumss#38)] -Aggregate Attributes [1]: [sum(sumss#38)#45] -Results [3]: [item_sk#33, d_date#29, sum(sumss#38)#45 AS cume_sales#46] - -(49) Exchange -Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: hashpartitioning(item_sk#33, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(50) Sort [codegen id : 24] -Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#33 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 - -(51) SortMergeJoin [codegen id : 25] -Left keys [2]: [item_sk#10, d_date#6] -Right keys [2]: [item_sk#33, d_date#29] +Join condition: (rk#36 >= rk#41) + +(48) Project [codegen id : 18] +Output [4]: [item_sk#34, d_date#30, sumss#35, sumss#39] +Input [7]: [item_sk#34, d_date#30, sumss#35, rk#36, item_sk#37, sumss#39, rk#41] + +(49) HashAggregate [codegen id : 18] +Input [4]: [item_sk#34, d_date#30, sumss#35, sumss#39] +Keys [3]: [item_sk#34, d_date#30, sumss#35] +Functions [1]: [partial_sum(sumss#39)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [5]: [item_sk#34, d_date#30, sumss#35, sum#44, isEmpty#45] + +(50) Exchange +Input [5]: [item_sk#34, d_date#30, sumss#35, sum#44, isEmpty#45] +Arguments: hashpartitioning(item_sk#34, d_date#30, sumss#35, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(51) HashAggregate [codegen id : 19] +Input [5]: [item_sk#34, d_date#30, sumss#35, sum#44, isEmpty#45] +Keys [3]: [item_sk#34, d_date#30, sumss#35] +Functions [1]: [sum(sumss#39)] +Aggregate Attributes [1]: [sum(sumss#39)#46] +Results [3]: [item_sk#34, d_date#30, sum(sumss#39)#46 AS cume_sales#47] + +(52) Exchange +Input [3]: [item_sk#34, d_date#30, cume_sales#47] +Arguments: hashpartitioning(item_sk#34, d_date#30, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(53) Sort [codegen id : 20] +Input [3]: [item_sk#34, d_date#30, cume_sales#47] +Arguments: [item_sk#34 ASC NULLS FIRST, d_date#30 ASC NULLS FIRST], false, 0 + +(54) SortMergeJoin [codegen id : 21] +Left keys [2]: [item_sk#11, d_date#6] +Right keys [2]: [item_sk#34, d_date#30] Join type: FullOuter Join condition: None -(52) Filter [codegen id : 25] -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END) +(55) Filter [codegen id : 21] +Input [6]: [item_sk#11, d_date#6, cume_sales#24, item_sk#34, d_date#30, cume_sales#47] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#11) THEN item_sk#11 ELSE item_sk#34 END) -(53) Project [codegen id : 25] -Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END AS item_sk#47, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#46 AS store_sales#50] -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] +(56) Project [codegen id : 21] +Output [4]: [CASE WHEN isnotnull(item_sk#11) THEN item_sk#11 ELSE item_sk#34 END AS item_sk#48, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#30 END AS d_date#49, cume_sales#24 AS web_sales#50, cume_sales#47 AS store_sales#51] +Input [6]: [item_sk#11, d_date#6, cume_sales#24, item_sk#34, d_date#30, cume_sales#47] -(54) Exchange -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: hashpartitioning(item_sk#47, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(57) Exchange +Input [4]: [item_sk#48, d_date#49, web_sales#50, store_sales#51] +Arguments: hashpartitioning(item_sk#48, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(55) Sort [codegen id : 26] -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], false, 0 +(58) Sort [codegen id : 22] +Input [4]: [item_sk#48, d_date#49, web_sales#50, store_sales#51] +Arguments: [item_sk#48 ASC NULLS FIRST, d_date#49 ASC NULLS FIRST], false, 0 -(56) Window -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#47], [d_date#48 ASC NULLS FIRST] +(59) Window +Input [4]: [item_sk#48, d_date#49, web_sales#50, store_sales#51] +Arguments: [row_number() windowspecdefinition(item_sk#48, d_date#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#52], [item_sk#48], [d_date#49 ASC NULLS FIRST] -(57) ReusedExchange [Reuses operator id: 54] -Output [4]: [item_sk#52, d_date#53, web_sales#54, store_sales#55] +(60) ReusedExchange [Reuses operator id: 57] +Output [4]: [item_sk#53, d_date#54, web_sales#55, store_sales#56] -(58) Sort [codegen id : 52] -Input [4]: [item_sk#52, d_date#53, web_sales#54, store_sales#55] -Arguments: [item_sk#52 ASC NULLS FIRST, d_date#53 ASC NULLS FIRST], false, 0 +(61) Sort [codegen id : 44] +Input [4]: [item_sk#53, d_date#54, web_sales#55, store_sales#56] +Arguments: [item_sk#53 ASC NULLS FIRST, d_date#54 ASC NULLS FIRST], false, 0 -(59) Window -Input [4]: [item_sk#52, d_date#53, web_sales#54, store_sales#55] -Arguments: [row_number() windowspecdefinition(item_sk#52, d_date#53 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#56], [item_sk#52], [d_date#53 ASC NULLS FIRST] +(62) Window +Input [4]: [item_sk#53, d_date#54, web_sales#55, store_sales#56] +Arguments: [row_number() windowspecdefinition(item_sk#53, d_date#54 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#57], [item_sk#53], [d_date#54 ASC NULLS FIRST] -(60) Project [codegen id : 53] -Output [4]: [item_sk#52, web_sales#54, store_sales#55, rk#56] -Input [5]: [item_sk#52, d_date#53, web_sales#54, store_sales#55, rk#56] +(63) Project [codegen id : 45] +Output [4]: [item_sk#53, web_sales#55, store_sales#56, rk#57] +Input [5]: [item_sk#53, d_date#54, web_sales#55, store_sales#56, rk#57] -(61) BroadcastExchange -Input [4]: [item_sk#52, web_sales#54, store_sales#55, rk#56] +(64) BroadcastExchange +Input [4]: [item_sk#53, web_sales#55, store_sales#56, rk#57] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(62) BroadcastHashJoin [codegen id : 54] -Left keys [1]: [item_sk#47] -Right keys [1]: [item_sk#52] +(65) BroadcastHashJoin [codegen id : 46] +Left keys [1]: [item_sk#48] +Right keys [1]: [item_sk#53] Join type: Inner -Join condition: (rk#51 >= rk#56) - -(63) Project [codegen id : 54] -Output [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] -Input [9]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#51, item_sk#52, web_sales#54, store_sales#55, rk#56] - -(64) HashAggregate [codegen id : 54] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] -Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Functions [2]: [partial_max(web_sales#54), partial_max(store_sales#55)] -Aggregate Attributes [2]: [max#57, max#58] -Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#59, max#60] - -(65) HashAggregate [codegen id : 54] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#59, max#60] -Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Functions [2]: [max(web_sales#54), max(store_sales#55)] -Aggregate Attributes [2]: [max(web_sales#54)#61, max(store_sales#55)#62] -Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max(web_sales#54)#61 AS web_cumulative#63, max(store_sales#55)#62 AS store_cumulative#64] - -(66) Filter [codegen id : 54] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#63, store_cumulative#64] -Condition : ((isnotnull(web_cumulative#63) AND isnotnull(store_cumulative#64)) AND (web_cumulative#63 > store_cumulative#64)) - -(67) TakeOrderedAndProject -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#63, store_cumulative#64] -Arguments: 100, [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#63, store_cumulative#64] +Join condition: (rk#52 >= rk#57) + +(66) Project [codegen id : 46] +Output [6]: [item_sk#48, d_date#49, web_sales#50, store_sales#51, web_sales#55, store_sales#56] +Input [9]: [item_sk#48, d_date#49, web_sales#50, store_sales#51, rk#52, item_sk#53, web_sales#55, store_sales#56, rk#57] + +(67) HashAggregate [codegen id : 46] +Input [6]: [item_sk#48, d_date#49, web_sales#50, store_sales#51, web_sales#55, store_sales#56] +Keys [4]: [item_sk#48, d_date#49, web_sales#50, store_sales#51] +Functions [2]: [partial_max(web_sales#55), partial_max(store_sales#56)] +Aggregate Attributes [2]: [max#58, max#59] +Results [6]: [item_sk#48, d_date#49, web_sales#50, store_sales#51, max#60, max#61] + +(68) HashAggregate [codegen id : 46] +Input [6]: [item_sk#48, d_date#49, web_sales#50, store_sales#51, max#60, max#61] +Keys [4]: [item_sk#48, d_date#49, web_sales#50, store_sales#51] +Functions [2]: [max(web_sales#55), max(store_sales#56)] +Aggregate Attributes [2]: [max(web_sales#55)#62, max(store_sales#56)#63] +Results [6]: [item_sk#48, d_date#49, web_sales#50, store_sales#51, max(web_sales#55)#62 AS web_cumulative#64, max(store_sales#56)#63 AS store_cumulative#65] + +(69) Filter [codegen id : 46] +Input [6]: [item_sk#48, d_date#49, web_sales#50, store_sales#51, web_cumulative#64, store_cumulative#65] +Condition : ((isnotnull(web_cumulative#64) AND isnotnull(store_cumulative#65)) AND (web_cumulative#64 > store_cumulative#65)) + +(70) TakeOrderedAndProject +Input [6]: [item_sk#48, d_date#49, web_sales#50, store_sales#51, web_cumulative#64, store_cumulative#65] +Arguments: 100, [item_sk#48 ASC NULLS FIRST, d_date#49 ASC NULLS FIRST], [item_sk#48, d_date#49, web_sales#50, store_sales#51, web_cumulative#64, store_cumulative#65] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) -(68) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#65] +(71) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#65] -Condition : (((isnotnull(d_month_seq#65) AND (d_month_seq#65 >= 1212)) AND (d_month_seq#65 <= 1223)) AND isnotnull(d_date_sk#5)) +(72) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(70) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#65] +(73) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(71) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(72) BroadcastExchange +(75) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#27 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt index 3109290dc7..892a3fc788 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (54) + WholeStageCodegen (46) Filter [web_cumulative,store_cumulative] HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] @@ -7,44 +7,44 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (26) + WholeStageCodegen (22) Sort [item_sk,d_date] InputAdapter Exchange [item_sk] #1 - WholeStageCodegen (25) + WholeStageCodegen (21) Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] Filter [item_sk,item_sk] SortMergeJoin [item_sk,d_date,item_sk,d_date] InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (10) Sort [item_sk,d_date] InputAdapter Exchange [item_sk,d_date] #2 - WholeStageCodegen (11) + WholeStageCodegen (9) HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] InputAdapter Exchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) + WholeStageCodegen (8) HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] Project [item_sk,d_date,sumws,sumws] BroadcastHashJoin [item_sk,item_sk,rk,rk] Project [item_sk,d_date,sumws,rk] InputAdapter Window [ws_item_sk,d_date] - WholeStageCodegen (4) + WholeStageCodegen (3) Sort [ws_item_sk,d_date] InputAdapter Exchange [ws_item_sk] #4 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] InputAdapter Exchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -55,70 +55,71 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometProject [d_date_sk,d_date] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + CometBroadcastExchange #7 + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) + BroadcastExchange #8 + WholeStageCodegen (7) Project [item_sk,sumws,rk] InputAdapter Window [ws_item_sk,d_date] - WholeStageCodegen (8) + WholeStageCodegen (6) Sort [ws_item_sk,d_date] InputAdapter ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (20) Sort [item_sk,d_date] InputAdapter - Exchange [item_sk,d_date] #8 - WholeStageCodegen (23) + Exchange [item_sk,d_date] #9 + WholeStageCodegen (19) HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] InputAdapter - Exchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (22) + Exchange [item_sk,d_date,sumss] #10 + WholeStageCodegen (18) HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] Project [item_sk,d_date,sumss,sumss] BroadcastHashJoin [item_sk,item_sk,rk,rk] Project [item_sk,d_date,sumss,rk] InputAdapter Window [ss_item_sk,d_date] - WholeStageCodegen (16) + WholeStageCodegen (13) Sort [ss_item_sk,d_date] InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (15) + Exchange [ss_item_sk] #11 + WholeStageCodegen (12) HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] InputAdapter - Exchange [ss_item_sk,d_date] #11 - WholeStageCodegen (14) + Exchange [ss_item_sk,d_date] #12 + WholeStageCodegen (11) HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + ReusedExchange [d_date_sk,d_date] #7 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (21) + BroadcastExchange #13 + WholeStageCodegen (17) Project [item_sk,sumss,rk] InputAdapter Window [ss_item_sk,d_date] - WholeStageCodegen (20) + WholeStageCodegen (16) Sort [ss_item_sk,d_date] InputAdapter - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (53) + BroadcastExchange #14 + WholeStageCodegen (45) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (52) + WholeStageCodegen (44) Sort [item_sk,d_date] InputAdapter ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt index dd64cc7bc8..e361df2d1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt @@ -14,22 +14,22 @@ TakeOrderedAndProject (45) : : +- * HashAggregate (21) : : +- Exchange (20) : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) + : : +- * ColumnarToRow (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.call_center (13) : +- BroadcastExchange (35) @@ -57,10 +57,7 @@ ReadSchema: struct Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] - -(4) Scan parquet spark_catalog.default.catalog_sales +(3) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -68,39 +65,46 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#7), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] Condition : (isnotnull(cs_item_sk#5) AND isnotnull(cs_call_center_sk#4)) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +Arguments: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Right output [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Arguments: [i_item_sk#1], [cs_item_sk#5], Inner, BuildRight -(9) Project [codegen id : 4] -Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] +(7) CometProject Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Arguments: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7], [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_year#10, d_moy#11] + +(11) CometBroadcastHashJoin +Left output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] +Right output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [cs_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(12) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] +(12) CometProject Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] +Arguments: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11], [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] (13) Scan parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#12, cc_name#13] @@ -113,24 +117,23 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#12, cc_name#13] Condition : (isnotnull(cc_call_center_sk#12) AND isnotnull(cc_name#13)) -(15) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [2]: [cc_call_center_sk#12, cc_name#13] +Arguments: [cc_call_center_sk#12, cc_name#13] -(16) BroadcastExchange -Input [2]: [cc_call_center_sk#12, cc_name#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] +Right output [2]: [cc_call_center_sk#12, cc_name#13] +Arguments: [cs_call_center_sk#4], [cc_call_center_sk#12], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#4] -Right keys [1]: [cc_call_center_sk#12] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] +(17) CometProject Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11, cc_call_center_sk#12, cc_name#13] +Arguments: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13], [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] + +(18) ColumnarToRow [codegen id : 1] +Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 1] Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#6))] @@ -139,9 +142,9 @@ Results [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] (20) Exchange Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 2] Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] @@ -150,9 +153,9 @@ Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDeci (22) Exchange Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(23) Sort [codegen id : 6] +(23) Sort [codegen id : 3] Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 @@ -160,7 +163,7 @@ Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] +(25) Filter [codegen id : 4] Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) @@ -168,18 +171,18 @@ Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] -(27) Filter [codegen id : 22] +(27) Filter [codegen id : 13] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(28) Project [codegen id : 22] +(28) Project [codegen id : 13] Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] (29) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(30) HashAggregate [codegen id : 12] +(30) HashAggregate [codegen id : 6] Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] @@ -188,9 +191,9 @@ Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDe (31) Exchange Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 13] +(32) Sort [codegen id : 7] Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 @@ -198,28 +201,28 @@ Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#2 Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(34) Project [codegen id : 14] +(34) Project [codegen id : 8] Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] (35) BroadcastExchange Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 22] +(36) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] +(37) Project [codegen id : 13] Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] (38) ReusedExchange [Reuses operator id: 31] Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(39) Sort [codegen id : 20] +(39) Sort [codegen id : 11] Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 @@ -227,21 +230,21 @@ Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#3 Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(41) Project [codegen id : 21] +(41) Project [codegen id : 12] Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] (42) BroadcastExchange Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 22] +(43) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] +(44) Project [codegen id : 13] Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] @@ -251,7 +254,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#1 ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (49) +- * ColumnarToRow (48) +- CometFilter (47) @@ -274,6 +277,6 @@ Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt index 56e33be9e4..b038aedf01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,74 +8,67 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter Exchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,cs_item_sk] CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange #3 + CometFilter [cs_item_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) + BroadcastExchange #7 + WholeStageCodegen (8) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) + WholeStageCodegen (7) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) + Exchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (6) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] InputAdapter ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #9 + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) + WholeStageCodegen (11) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt index 2769e772ae..9d660d776a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt @@ -8,68 +8,68 @@ TakeOrderedAndProject (83) : +- Exchange (67) : +- * HashAggregate (66) : +- Union (65) - : :- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (11) - : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : :- * ColumnarToRow (8) - : : : : +- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : +- ReusedExchange (9) - : : +- BroadcastExchange (15) - : : +- * ColumnarToRow (14) - : : +- CometFilter (13) - : : +- CometScan parquet spark_catalog.default.store (12) - : :- * HashAggregate (40) - : : +- Exchange (39) - : : +- * HashAggregate (38) - : : +- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * ColumnarToRow (28) - : : : : +- CometUnion (27) - : : : : :- CometProject (23) - : : : : : +- CometFilter (22) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) - : : : : +- CometProject (26) - : : : : +- CometFilter (25) - : : : : +- CometScan parquet spark_catalog.default.catalog_returns (24) - : : : +- ReusedExchange (29) - : : +- BroadcastExchange (35) - : : +- * ColumnarToRow (34) - : : +- CometFilter (33) - : : +- CometScan parquet spark_catalog.default.catalog_page (32) + : :- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * ColumnarToRow (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometProject (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.store (14) + : :- * HashAggregate (41) + : : +- Exchange (40) + : : +- * HashAggregate (39) + : : +- * ColumnarToRow (38) + : : +- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (30) + : : +- CometBroadcastExchange (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.catalog_page (33) : +- * HashAggregate (64) : +- Exchange (63) : +- * HashAggregate (62) - : +- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Project (55) - : : +- * BroadcastHashJoin Inner BuildRight (54) - : : :- * ColumnarToRow (52) - : : : +- CometUnion (51) - : : : :- CometProject (43) - : : : : +- CometFilter (42) - : : : : +- CometScan parquet spark_catalog.default.web_sales (41) - : : : +- CometProject (50) - : : : +- CometBroadcastHashJoin (49) - : : : :- CometBroadcastExchange (45) - : : : : +- CometScan parquet spark_catalog.default.web_returns (44) - : : : +- CometProject (48) - : : : +- CometFilter (47) - : : : +- CometScan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) + : +- * ColumnarToRow (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometUnion (52) + : : : :- CometProject (44) + : : : : +- CometFilter (43) + : : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : : +- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometBroadcastExchange (46) + : : : : +- CometScan parquet spark_catalog.default.web_returns (45) + : : : +- CometProject (49) + : : : +- CometFilter (48) + : : : +- CometScan parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (53) + : +- CometBroadcastExchange (58) : +- CometFilter (57) : +- CometScan parquet spark_catalog.default.web_site (56) :- * HashAggregate (73) @@ -120,382 +120,383 @@ Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, n Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] -(8) ColumnarToRow [codegen id : 3] -Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] +ReadSchema: struct + +(9) CometFilter +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(9) ReusedExchange [Reuses operator id: 88] -Output [1]: [d_date_sk#22] +(10) CometProject +Input [2]: [d_date_sk#22, d_date#23] +Arguments: [d_date_sk#22], [d_date_sk#22] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: [d_date_sk#22] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [1]: [d_date_sk#22] +Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight -(11) Project [codegen id : 3] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +(13) CometProject Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] +Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -(12) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] +(14) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_store_id#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(13) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) +(15) CometFilter +Input [2]: [s_store_sk#24, s_store_id#25] +Condition : isnotnull(s_store_sk#24) -(14) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#23, s_store_id#24] +(16) CometBroadcastExchange +Input [2]: [s_store_sk#24, s_store_id#25] +Arguments: [s_store_sk#24, s_store_id#25] -(15) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(17) CometBroadcastHashJoin +Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [2]: [s_store_sk#24, s_store_id#25] +Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None +(18) CometProject +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] +Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -(17) Project [codegen id : 3] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#24] +(19) ColumnarToRow [codegen id : 1] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -(18) HashAggregate [codegen id : 3] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Keys [1]: [s_store_id#24] +(20) HashAggregate [codegen id : 1] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] -Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] +Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(19) Exchange -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(21) Exchange +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Keys [1]: [s_store_id#24] +(22) HashAggregate [codegen id : 2] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] +Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] -Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] +Results [5]: [store channel AS channel#38, concat(store, s_store_id#25) AS id#39, MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#41, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#42] -(21) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(23) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#47)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(22) CometFilter -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : isnotnull(cs_catalog_page_sk#42) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Condition : isnotnull(cs_catalog_page_sk#43) -(23) CometProject -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52], [cs_catalog_page_sk#42 AS page_sk#47, cs_sold_date_sk#45 AS date_sk#48, cs_ext_sales_price#43 AS sales_price#49, cs_net_profit#44 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] +(25) CometProject +Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] +Arguments: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53], [cs_catalog_page_sk#43 AS page_sk#48, cs_sold_date_sk#46 AS date_sk#49, cs_ext_sales_price#44 AS sales_price#50, cs_net_profit#45 AS profit#51, 0.00 AS return_amt#52, 0.00 AS net_loss#53] -(24) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +(26) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#47)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Condition : isnotnull(cr_catalog_page_sk#53) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Condition : isnotnull(cr_catalog_page_sk#54) -(26) CometProject -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Arguments: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62], [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] +(28) CometProject +Input [4]: [cr_catalog_page_sk#54, cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Arguments: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63], [cr_catalog_page_sk#54 AS page_sk#58, cr_returned_date_sk#57 AS date_sk#59, 0.00 AS sales_price#60, 0.00 AS profit#61, cr_return_amount#55 AS return_amt#62, cr_net_loss#56 AS net_loss#63] -(27) CometUnion -Child 0 Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] -Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62] +(29) CometUnion +Child 0 Input [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] +Child 1 Input [6]: [page_sk#58, date_sk#59, sales_price#60, profit#61, return_amt#62, net_loss#63] -(28) ColumnarToRow [codegen id : 7] -Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#64] -(29) ReusedExchange [Reuses operator id: 88] -Output [1]: [d_date_sk#63] +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53] +Right output [1]: [d_date_sk#64] +Arguments: [date_sk#49], [d_date_sk#64], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [date_sk#48] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None +(32) CometProject +Input [7]: [page_sk#48, date_sk#49, sales_price#50, profit#51, return_amt#52, net_loss#53, d_date_sk#64] +Arguments: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53], [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53] -(31) Project [codegen id : 7] -Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] -Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] - -(32) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +(33) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(33) CometFilter -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) - -(34) ColumnarToRow [codegen id : 6] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] - -(35) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 7] -Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] -Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#65] - -(38) HashAggregate [codegen id : 7] -Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] -Keys [1]: [cp_catalog_page_id#65] -Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum#66, sum#67, sum#68, sum#69] -Results [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] - -(39) Exchange -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(40) HashAggregate [codegen id : 8] -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Keys [1]: [cp_catalog_page_id#65] -Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] -Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#65) AS id#79, MakeDecimal(sum(UnscaledValue(sales_price#49))#74,17,2) AS sales#80, MakeDecimal(sum(UnscaledValue(return_amt#51))#75,17,2) AS returns#81, (MakeDecimal(sum(UnscaledValue(profit#50))#76,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#77,17,2)) AS profit#82] - -(41) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +(34) CometFilter +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Condition : isnotnull(cp_catalog_page_sk#65) + +(35) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Arguments: [cp_catalog_page_sk#65, cp_catalog_page_id#66] + +(36) CometBroadcastHashJoin +Left output [5]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53] +Right output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] +Arguments: [page_sk#48], [cp_catalog_page_sk#65], Inner, BuildRight + +(37) CometProject +Input [7]: [page_sk#48, sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_sk#65, cp_catalog_page_id#66] +Arguments: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66], [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] + +(38) ColumnarToRow [codegen id : 3] +Input [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] + +(39) HashAggregate [codegen id : 3] +Input [5]: [sales_price#50, profit#51, return_amt#52, net_loss#53, cp_catalog_page_id#66] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [partial_sum(UnscaledValue(sales_price#50)), partial_sum(UnscaledValue(return_amt#52)), partial_sum(UnscaledValue(profit#51)), partial_sum(UnscaledValue(net_loss#53))] +Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] +Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] + +(40) Exchange +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(41) HashAggregate [codegen id : 4] +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] +Keys [1]: [cp_catalog_page_id#66] +Functions [4]: [sum(UnscaledValue(sales_price#50)), sum(UnscaledValue(return_amt#52)), sum(UnscaledValue(profit#51)), sum(UnscaledValue(net_loss#53))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#50))#75, sum(UnscaledValue(return_amt#52))#76, sum(UnscaledValue(profit#51))#77, sum(UnscaledValue(net_loss#53))#78] +Results [5]: [catalog channel AS channel#79, concat(catalog_page, cp_catalog_page_id#66) AS id#80, MakeDecimal(sum(UnscaledValue(sales_price#50))#75,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#52))#76,17,2) AS returns#82, (MakeDecimal(sum(UnscaledValue(profit#51))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#53))#78,17,2)) AS profit#83] + +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(42) CometFilter -Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] -Condition : isnotnull(ws_web_site_sk#83) +(43) CometFilter +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] +Condition : isnotnull(ws_web_site_sk#84) -(43) CometProject -Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] -Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] +(44) CometProject +Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] +Arguments: [wsr_web_site_sk#89, date_sk#90, sales_price#91, profit#92, return_amt#93, net_loss#94], [ws_web_site_sk#84 AS wsr_web_site_sk#89, ws_sold_date_sk#87 AS date_sk#90, ws_ext_sales_price#85 AS sales_price#91, ws_net_profit#86 AS profit#92, 0.00 AS return_amt#93, 0.00 AS net_loss#94] -(44) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +(45) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(wr_returned_date_sk#99), dynamicpruningexpression(wr_returned_date_sk#99 IN dynamicpruning#88)] ReadSchema: struct -(45) CometBroadcastExchange -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] +Arguments: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] -(46) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +(47) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(47) CometFilter -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) - -(48) CometProject -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +(48) CometFilter +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] +Condition : ((isnotnull(ws_item_sk#100) AND isnotnull(ws_order_number#102)) AND isnotnull(ws_web_site_sk#101)) -(49) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner +(49) CometProject +Input [4]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102, ws_sold_date_sk#103] +Arguments: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102], [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] -(50) CometProject -Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99] +Right output [3]: [ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Arguments: [wr_item_sk#95, wr_order_number#96], [ws_item_sk#100, ws_order_number#102], Inner, BuildLeft -(51) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] +(51) CometProject +Input [8]: [wr_item_sk#95, wr_order_number#96, wr_return_amt#97, wr_net_loss#98, wr_returned_date_sk#99, ws_item_sk#100, ws_web_site_sk#101, ws_order_number#102] +Arguments: [wsr_web_site_sk#104, date_sk#105, sales_price#106, profit#107, return_amt#108, net_loss#109], [ws_web_site_sk#101 AS wsr_web_site_sk#104, wr_returned_date_sk#99 AS date_sk#105, 0.00 AS sales_price#106, 0.00 AS profit#107, wr_return_amt#97 AS return_amt#108, wr_net_loss#98 AS net_loss#109] -(52) ColumnarToRow [codegen id : 11] -Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#89, date_sk#90, sales_price#91, profit#92, return_amt#93, net_loss#94] +Child 1 Input [6]: [wsr_web_site_sk#104, date_sk#105, sales_price#106, profit#107, return_amt#108, net_loss#109] -(53) ReusedExchange [Reuses operator id: 88] -Output [1]: [d_date_sk#109] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#110] -(54) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#89] -Right keys [1]: [d_date_sk#109] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#89, date_sk#90, sales_price#91, profit#92, return_amt#93, net_loss#94] +Right output [1]: [d_date_sk#110] +Arguments: [date_sk#90], [d_date_sk#110], Inner, BuildRight -(55) Project [codegen id : 11] -Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] -Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] +(55) CometProject +Input [7]: [wsr_web_site_sk#89, date_sk#90, sales_price#91, profit#92, return_amt#93, net_loss#94, d_date_sk#110] +Arguments: [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94], [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94] (56) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#110, web_site_id#111] +Output [2]: [web_site_sk#111, web_site_id#112] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (57) CometFilter -Input [2]: [web_site_sk#110, web_site_id#111] -Condition : isnotnull(web_site_sk#110) +Input [2]: [web_site_sk#111, web_site_id#112] +Condition : isnotnull(web_site_sk#111) -(58) ColumnarToRow [codegen id : 10] -Input [2]: [web_site_sk#110, web_site_id#111] +(58) CometBroadcastExchange +Input [2]: [web_site_sk#111, web_site_id#112] +Arguments: [web_site_sk#111, web_site_id#112] -(59) BroadcastExchange -Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(59) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94] +Right output [2]: [web_site_sk#111, web_site_id#112] +Arguments: [wsr_web_site_sk#89], [web_site_sk#111], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [wsr_web_site_sk#88] -Right keys [1]: [web_site_sk#110] -Join type: Inner -Join condition: None +(60) CometProject +Input [7]: [wsr_web_site_sk#89, sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_sk#111, web_site_id#112] +Arguments: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112], [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] -(61) Project [codegen id : 11] -Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] -Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] +(61) ColumnarToRow [codegen id : 5] +Input [5]: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] -(62) HashAggregate [codegen id : 11] -Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] -Keys [1]: [web_site_id#111] -Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] -Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] +(62) HashAggregate [codegen id : 5] +Input [5]: [sales_price#91, profit#92, return_amt#93, net_loss#94, web_site_id#112] +Keys [1]: [web_site_id#112] +Functions [4]: [partial_sum(UnscaledValue(sales_price#91)), partial_sum(UnscaledValue(return_amt#93)), partial_sum(UnscaledValue(profit#92)), partial_sum(UnscaledValue(net_loss#94))] +Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] +Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] (63) Exchange -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(64) HashAggregate [codegen id : 12] -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Keys [1]: [web_site_id#111] -Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] -Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] +(64) HashAggregate [codegen id : 6] +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] +Keys [1]: [web_site_id#112] +Functions [4]: [sum(UnscaledValue(sales_price#91)), sum(UnscaledValue(return_amt#93)), sum(UnscaledValue(profit#92)), sum(UnscaledValue(net_loss#94))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#91))#121, sum(UnscaledValue(return_amt#93))#122, sum(UnscaledValue(profit#92))#123, sum(UnscaledValue(net_loss#94))#124] +Results [5]: [web channel AS channel#125, concat(web_site, web_site_id#112) AS id#126, MakeDecimal(sum(UnscaledValue(sales_price#91))#121,17,2) AS sales#127, MakeDecimal(sum(UnscaledValue(return_amt#93))#122,17,2) AS returns#128, (MakeDecimal(sum(UnscaledValue(profit#92))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#94))#124,17,2)) AS profit#129] (65) Union -(66) HashAggregate [codegen id : 13] -Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] -Keys [2]: [channel#37, id#38] -Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] -Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +(66) HashAggregate [codegen id : 7] +Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] +Keys [2]: [channel#38, id#39] +Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] +Aggregate Attributes [6]: [sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] +Results [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] (67) Exchange -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] +Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(68) HashAggregate [codegen id : 14] -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] -Results [5]: [channel#37, id#38, cast(sum(sales#39)#141 as decimal(37,2)) AS sales#144, cast(sum(returns#40)#142 as decimal(37,2)) AS returns#145, cast(sum(profit#41)#143 as decimal(38,2)) AS profit#146] +(68) HashAggregate [codegen id : 8] +Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] +Keys [2]: [channel#38, id#39] +Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] +Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] +Results [5]: [channel#38, id#39, cast(sum(sales#40)#142 as decimal(37,2)) AS sales#145, cast(sum(returns#41)#143 as decimal(37,2)) AS returns#146, cast(sum(profit#42)#144 as decimal(38,2)) AS profit#147] (69) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#147, id#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] - -(70) HashAggregate [codegen id : 28] -Input [8]: [channel#147, id#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] -Keys [2]: [channel#147, id#148] -Functions [3]: [sum(sales#155), sum(returns#156), sum(profit#157)] -Aggregate Attributes [3]: [sum(sales#155)#141, sum(returns#156)#142, sum(profit#157)#143] -Results [4]: [channel#147, sum(sales#155)#141 AS sales#158, sum(returns#156)#142 AS returns#159, sum(profit#157)#143 AS profit#160] - -(71) HashAggregate [codegen id : 28] -Input [4]: [channel#147, sales#158, returns#159, profit#160] -Keys [1]: [channel#147] -Functions [3]: [partial_sum(sales#158), partial_sum(returns#159), partial_sum(profit#160)] -Aggregate Attributes [6]: [sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166] -Results [7]: [channel#147, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] +Output [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] + +(70) HashAggregate [codegen id : 16] +Input [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] +Keys [2]: [channel#148, id#149] +Functions [3]: [sum(sales#156), sum(returns#157), sum(profit#158)] +Aggregate Attributes [3]: [sum(sales#156)#142, sum(returns#157)#143, sum(profit#158)#144] +Results [4]: [channel#148, sum(sales#156)#142 AS sales#159, sum(returns#157)#143 AS returns#160, sum(profit#158)#144 AS profit#161] + +(71) HashAggregate [codegen id : 16] +Input [4]: [channel#148, sales#159, returns#160, profit#161] +Keys [1]: [channel#148] +Functions [3]: [partial_sum(sales#159), partial_sum(returns#160), partial_sum(profit#161)] +Aggregate Attributes [6]: [sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Results [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] (72) Exchange -Input [7]: [channel#147, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] -Arguments: hashpartitioning(channel#147, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] +Arguments: hashpartitioning(channel#148, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(73) HashAggregate [codegen id : 29] -Input [7]: [channel#147, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] -Keys [1]: [channel#147] -Functions [3]: [sum(sales#158), sum(returns#159), sum(profit#160)] -Aggregate Attributes [3]: [sum(sales#158)#173, sum(returns#159)#174, sum(profit#160)#175] -Results [5]: [channel#147, null AS id#176, sum(sales#158)#173 AS sum(sales)#177, sum(returns#159)#174 AS sum(returns)#178, sum(profit#160)#175 AS sum(profit)#179] +(73) HashAggregate [codegen id : 17] +Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] +Keys [1]: [channel#148] +Functions [3]: [sum(sales#159), sum(returns#160), sum(profit#161)] +Aggregate Attributes [3]: [sum(sales#159)#174, sum(returns#160)#175, sum(profit#161)#176] +Results [5]: [channel#148, null AS id#177, sum(sales#159)#174 AS sum(sales)#178, sum(returns#160)#175 AS sum(returns)#179, sum(profit#161)#176 AS sum(profit)#180] (74) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#180, id#181, sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] +Output [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -(75) HashAggregate [codegen id : 43] -Input [8]: [channel#180, id#181, sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] -Keys [2]: [channel#180, id#181] -Functions [3]: [sum(sales#188), sum(returns#189), sum(profit#190)] -Aggregate Attributes [3]: [sum(sales#188)#141, sum(returns#189)#142, sum(profit#190)#143] -Results [3]: [sum(sales#188)#141 AS sales#191, sum(returns#189)#142 AS returns#192, sum(profit#190)#143 AS profit#193] +(75) HashAggregate [codegen id : 25] +Input [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] +Keys [2]: [channel#181, id#182] +Functions [3]: [sum(sales#189), sum(returns#190), sum(profit#191)] +Aggregate Attributes [3]: [sum(sales#189)#142, sum(returns#190)#143, sum(profit#191)#144] +Results [3]: [sum(sales#189)#142 AS sales#192, sum(returns#190)#143 AS returns#193, sum(profit#191)#144 AS profit#194] -(76) HashAggregate [codegen id : 43] -Input [3]: [sales#191, returns#192, profit#193] +(76) HashAggregate [codegen id : 25] +Input [3]: [sales#192, returns#193, profit#194] Keys: [] -Functions [3]: [partial_sum(sales#191), partial_sum(returns#192), partial_sum(profit#193)] -Aggregate Attributes [6]: [sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199] -Results [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] +Functions [3]: [partial_sum(sales#192), partial_sum(returns#193), partial_sum(profit#194)] +Aggregate Attributes [6]: [sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200] +Results [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] (77) Exchange -Input [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(78) HashAggregate [codegen id : 44] -Input [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] +(78) HashAggregate [codegen id : 26] +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Keys: [] -Functions [3]: [sum(sales#191), sum(returns#192), sum(profit#193)] -Aggregate Attributes [3]: [sum(sales#191)#206, sum(returns#192)#207, sum(profit#193)#208] -Results [5]: [null AS channel#209, null AS id#210, sum(sales#191)#206 AS sum(sales)#211, sum(returns#192)#207 AS sum(returns)#212, sum(profit#193)#208 AS sum(profit)#213] +Functions [3]: [sum(sales#192), sum(returns#193), sum(profit#194)] +Aggregate Attributes [3]: [sum(sales#192)#207, sum(returns#193)#208, sum(profit#194)#209] +Results [5]: [null AS channel#210, null AS id#211, sum(sales#192)#207 AS sum(sales)#212, sum(returns#193)#208 AS sum(returns)#213, sum(profit#194)#209 AS sum(profit)#214] (79) Union -(80) HashAggregate [codegen id : 45] -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +(80) HashAggregate [codegen id : 27] +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Results [5]: [channel#38, id#39, sales#145, returns#146, profit#147] (81) Exchange -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Arguments: hashpartitioning(channel#37, id#38, sales#144, returns#145, profit#146, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Arguments: hashpartitioning(channel#38, id#39, sales#145, returns#146, profit#147, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(82) HashAggregate [codegen id : 46] -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +(82) HashAggregate [codegen id : 28] +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Results [5]: [channel#38, id#39, sales#145, returns#146, profit#147] (83) TakeOrderedAndProject -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#144, returns#145, profit#146] +Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] +Arguments: 100, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#145, returns#146, profit#147] ===== Subqueries ===== @@ -508,18 +509,18 @@ BroadcastExchange (88) (84) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#214] +Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct (85) CometFilter -Input [2]: [d_date_sk#22, d_date#214] -Condition : (((isnotnull(d_date#214) AND (d_date#214 >= 1998-08-04)) AND (d_date#214 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) (86) CometProject -Input [2]: [d_date_sk#22, d_date#214] +Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] (87) ColumnarToRow [codegen id : 1] @@ -527,16 +528,16 @@ Input [1]: [d_date_sk#22] (88) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#99 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt index aaec304fd0..34ad132c09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt @@ -1,32 +1,32 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (46) + WholeStageCodegen (28) HashAggregate [channel,id,sales,returns,profit] InputAdapter Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (45) + WholeStageCodegen (27) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (14) + WholeStageCodegen (8) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #2 - WholeStageCodegen (13) + WholeStageCodegen (7) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter Exchange [s_store_id] #3 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,s_store_sk] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [ss_store_sk] @@ -43,27 +43,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (8) + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter - Exchange [cp_catalog_page_id] #6 - WholeStageCodegen (7) + Exchange [cp_catalog_page_id] #7 + WholeStageCodegen (3) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [cs_catalog_page_sk] @@ -73,27 +71,22 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [cr_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (12) + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #8 + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter - Exchange [web_site_id] #8 - WholeStageCodegen (11) + Exchange [web_site_id] #9 + WholeStageCodegen (5) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [ws_web_site_sk] @@ -101,35 +94,30 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #9 + CometBroadcastExchange #10 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (29) + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #11 + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (17) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [channel] #11 - WholeStageCodegen (28) + Exchange [channel] #12 + WholeStageCodegen (16) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (44) + WholeStageCodegen (26) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange #12 - WholeStageCodegen (43) + Exchange #13 + WholeStageCodegen (25) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt index a71a3a875b..fbf8c8eada 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt @@ -1,43 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Filter (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer_address (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (29) - +- * Filter (28) - +- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * ColumnarToRow (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.item (22) +TakeOrderedAndProject (40) ++- * Filter (39) + +- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer_address (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.customer (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.store_sales (8) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.date_dim (13) + +- BroadcastExchange (33) + +- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- BroadcastExchange (30) + +- * Filter (29) + +- * HashAggregate (28) + +- Exchange (27) + +- * HashAggregate (26) + +- * ColumnarToRow (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.item (23) (1) Scan parquet spark_catalog.default.customer_address @@ -51,38 +52,31 @@ ReadSchema: struct Input [2]: [ca_address_sk#1, ca_state#2] Condition : isnotnull(ca_address_sk#1) -(3) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#1, ca_state#2] - -(4) Scan parquet spark_catalog.default.customer +(3) Scan parquet spark_catalog.default.customer Output [2]: [c_customer_sk#3, c_current_addr_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Condition : (isnotnull(c_current_addr_sk#4) AND isnotnull(c_customer_sk#3)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [2]: [c_customer_sk#3, c_current_addr_sk#4] +Arguments: [c_customer_sk#3, c_current_addr_sk#4] -(7) BroadcastExchange -Input [2]: [c_customer_sk#3, c_current_addr_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [2]: [ca_address_sk#1, ca_state#2] +Right output [2]: [c_customer_sk#3, c_current_addr_sk#4] +Arguments: [ca_address_sk#1], [c_current_addr_sk#4], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ca_address_sk#1] -Right keys [1]: [c_current_addr_sk#4] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 7] -Output [2]: [ca_state#2, c_customer_sk#3] +(7) CometProject Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4] +Arguments: [ca_state#2, c_customer_sk#3], [ca_state#2, c_customer_sk#3] -(10) Scan parquet spark_catalog.default.store_sales +(8) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -90,220 +84,238 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Condition : (isnotnull(ss_customer_sk#6) AND isnotnull(ss_item_sk#5)) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -(13) BroadcastExchange -Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [2]: [ca_state#2, c_customer_sk#3] +Right output [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 7] -Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] +(12) CometProject Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7], [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] -(16) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#9] +(13) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_month_seq#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#11), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(14) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#10] +Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date_sk#9)) + +(15) CometProject +Input [2]: [d_date_sk#9, d_month_seq#10] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(16) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] -(18) Project [codegen id : 7] -Output [2]: [ca_state#2, ss_item_sk#5] +(17) CometBroadcastHashJoin +Left output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight + +(18) CometProject Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ca_state#2, ss_item_sk#5], [ca_state#2, ss_item_sk#5] + +(19) ColumnarToRow [codegen id : 4] +Input [2]: [ca_state#2, ss_item_sk#5] -(19) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#10, i_current_price#11, i_category#12] +(20) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#13, i_current_price#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] ReadSchema: struct -(20) CometFilter -Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] -Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10)) +(21) CometFilter +Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] +Condition : ((isnotnull(i_current_price#14) AND isnotnull(i_category#15)) AND isnotnull(i_item_sk#13)) -(21) ColumnarToRow [codegen id : 6] -Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] +(22) ColumnarToRow [codegen id : 3] +Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] -(22) Scan parquet spark_catalog.default.item -Output [2]: [i_current_price#13, i_category#14] +(23) Scan parquet spark_catalog.default.item +Output [2]: [i_current_price#16, i_category#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct -(23) CometFilter -Input [2]: [i_current_price#13, i_category#14] -Condition : isnotnull(i_category#14) - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#13, i_category#14] - -(25) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#13, i_category#14] -Keys [1]: [i_category#14] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] -Aggregate Attributes [2]: [sum#15, count#16] -Results [3]: [i_category#14, sum#17, count#18] - -(26) Exchange -Input [3]: [i_category#14, sum#17, count#18] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 5] -Input [3]: [i_category#14, sum#17, count#18] -Keys [1]: [i_category#14] -Functions [1]: [avg(UnscaledValue(i_current_price#13))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#19] -Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#19 / 100.0) as decimal(11,6)) AS avg(i_current_price)#20, i_category#14] - -(28) Filter [codegen id : 5] -Input [2]: [avg(i_current_price)#20, i_category#14] -Condition : isnotnull(avg(i_current_price)#20) - -(29) BroadcastExchange -Input [2]: [avg(i_current_price)#20, i_category#14] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_category#12] -Right keys [1]: [i_category#14] +(24) CometFilter +Input [2]: [i_current_price#16, i_category#17] +Condition : isnotnull(i_category#17) + +(25) ColumnarToRow [codegen id : 1] +Input [2]: [i_current_price#16, i_category#17] + +(26) HashAggregate [codegen id : 1] +Input [2]: [i_current_price#16, i_category#17] +Keys [1]: [i_category#17] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#16))] +Aggregate Attributes [2]: [sum#18, count#19] +Results [3]: [i_category#17, sum#20, count#21] + +(27) Exchange +Input [3]: [i_category#17, sum#20, count#21] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(28) HashAggregate [codegen id : 2] +Input [3]: [i_category#17, sum#20, count#21] +Keys [1]: [i_category#17] +Functions [1]: [avg(UnscaledValue(i_current_price#16))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#16))#22] +Results [2]: [cast((avg(UnscaledValue(i_current_price#16))#22 / 100.0) as decimal(11,6)) AS avg(i_current_price)#23, i_category#17] + +(29) Filter [codegen id : 2] +Input [2]: [avg(i_current_price)#23, i_category#17] +Condition : isnotnull(avg(i_current_price)#23) + +(30) BroadcastExchange +Input [2]: [avg(i_current_price)#23, i_category#17] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=2] + +(31) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_category#15] +Right keys [1]: [i_category#17] Join type: Inner -Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)) +Join condition: (cast(i_current_price#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#23)) -(31) Project [codegen id : 6] -Output [1]: [i_item_sk#10] -Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#20, i_category#14] +(32) Project [codegen id : 3] +Output [1]: [i_item_sk#13] +Input [5]: [i_item_sk#13, i_current_price#14, i_category#15, avg(i_current_price)#23, i_category#17] -(32) BroadcastExchange -Input [1]: [i_item_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) BroadcastExchange +Input [1]: [i_item_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 7] +(34) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#10] +Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(34) Project [codegen id : 7] +(35) Project [codegen id : 4] Output [1]: [ca_state#2] -Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#10] +Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#13] -(35) HashAggregate [codegen id : 7] +(36) HashAggregate [codegen id : 4] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#21] -Results [2]: [ca_state#2, count#22] +Aggregate Attributes [1]: [count#24] +Results [2]: [ca_state#2, count#25] -(36) Exchange -Input [2]: [ca_state#2, count#22] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(37) Exchange +Input [2]: [ca_state#2, count#25] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(37) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#22] +(38) HashAggregate [codegen id : 5] +Input [2]: [ca_state#2, count#25] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [2]: [ca_state#2 AS state#24, count(1)#23 AS cnt#25] +Aggregate Attributes [1]: [count(1)#26] +Results [2]: [ca_state#2 AS state#27, count(1)#26 AS cnt#28] -(38) Filter [codegen id : 8] -Input [2]: [state#24, cnt#25] -Condition : (cnt#25 >= 10) +(39) Filter [codegen id : 5] +Input [2]: [state#27, cnt#28] +Condition : (cnt#28 >= 10) -(39) TakeOrderedAndProject -Input [2]: [state#24, cnt#25] -Arguments: 100, [cnt#25 ASC NULLS FIRST, state#24 ASC NULLS FIRST], [state#24, cnt#25] +(40) TakeOrderedAndProject +Input [2]: [state#27, cnt#28] +Arguments: 100, [cnt#28 ASC NULLS FIRST, state#27 ASC NULLS FIRST], [state#27, cnt#28] ===== Subqueries ===== -Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +Subquery:1 Hosting operator id = 8 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) -(40) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#26] +(41) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_month_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#27), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#11), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#26] -Condition : ((isnotnull(d_month_seq#26) AND (d_month_seq#26 = ReusedSubquery Subquery scalar-subquery#27, [id=#28])) AND isnotnull(d_date_sk#9)) +(42) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#10] +Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date_sk#9)) -(42) CometProject -Input [2]: [d_date_sk#9, d_month_seq#26] +(43) CometProject +Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(43) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(44) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 41 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] +Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -Subquery:3 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* HashAggregate (51) -+- Exchange (50) - +- * ColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* HashAggregate (52) ++- Exchange (51) + +- * ColumnarToRow (50) + +- CometHashAggregate (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(45) Scan parquet spark_catalog.default.date_dim +(46) Scan parquet spark_catalog.default.date_dim Output [3]: [d_month_seq#29, d_year#30, d_moy#31] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(46) CometFilter +(47) CometFilter Input [3]: [d_month_seq#29, d_year#30, d_moy#31] Condition : (((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 1)) -(47) CometProject +(48) CometProject Input [3]: [d_month_seq#29, d_year#30, d_moy#31] Arguments: [d_month_seq#29], [d_month_seq#29] -(48) CometHashAggregate +(49) CometHashAggregate Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -(49) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_month_seq#29] -(50) Exchange +(51) Exchange Input [1]: [d_month_seq#29] -Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(51) HashAggregate [codegen id : 2] +(52) HashAggregate [codegen id : 2] Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] Aggregate Attributes: [] Results [1]: [d_month_seq#29] +Subquery:4 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] + +Subquery:5 Hosting operator id = 13 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt index c2d5a6ce83..a7c65f50ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt @@ -1,63 +1,59 @@ TakeOrderedAndProject [cnt,state] - WholeStageCodegen (8) + WholeStageCodegen (5) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] InputAdapter Exchange [ca_state] #1 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - ReusedSubquery [d_month_seq] #2 - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (2) - HashAggregate [d_month_seq] - InputAdapter - Exchange [d_month_seq] #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #2 + CometFilter [c_current_addr_sk,c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #3 + CometFilter [ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (2) + HashAggregate [d_month_seq] + InputAdapter + Exchange [d_month_seq] #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (3) Project [i_item_sk] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow @@ -65,13 +61,13 @@ TakeOrderedAndProject [cnt,state] CometFilter [i_current_price,i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) + BroadcastExchange #8 + WholeStageCodegen (2) Filter [avg(i_current_price)] HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter - Exchange [i_category] #8 - WholeStageCodegen (4) + Exchange [i_category] #9 + WholeStageCodegen (1) HashAggregate [i_category,i_current_price] [sum,count,sum,count] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt index e50a522b80..b74018441d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt @@ -216,7 +216,7 @@ Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_numbe (7) CometBroadcastHashJoin Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner +Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft (8) CometProject Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] @@ -725,7 +725,7 @@ Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_nu (117) CometBroadcastHashJoin Left output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Right output [2]: [sr_item_sk#119, sr_ticket_number#120] -Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner +Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner, BuildLeft (118) CometProject Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt index 6ca27323ac..25e832248e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt @@ -1,74 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * Filter (69) - +- Window (68) - +- WindowGroupLimit (67) - +- * Sort (66) - +- Exchange (65) - +- WindowGroupLimit (64) - +- * Sort (63) - +- Union (62) - :- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.item (13) - :- * HashAggregate (26) - : +- Exchange (25) - : +- * HashAggregate (24) - : +- * HashAggregate (23) - : +- ReusedExchange (22) - :- * HashAggregate (31) - : +- Exchange (30) - : +- * HashAggregate (29) - : +- * HashAggregate (28) - : +- ReusedExchange (27) - :- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * HashAggregate (33) - : +- ReusedExchange (32) - :- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * HashAggregate (38) - : +- ReusedExchange (37) - :- * HashAggregate (46) - : +- Exchange (45) - : +- * HashAggregate (44) - : +- * HashAggregate (43) - : +- ReusedExchange (42) - :- * HashAggregate (51) - : +- Exchange (50) - : +- * HashAggregate (49) - : +- * HashAggregate (48) - : +- ReusedExchange (47) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * HashAggregate (53) - : +- ReusedExchange (52) - +- * HashAggregate (61) - +- Exchange (60) - +- * HashAggregate (59) - +- * HashAggregate (58) - +- ReusedExchange (57) +TakeOrderedAndProject (71) ++- * Filter (70) + +- Window (69) + +- WindowGroupLimit (68) + +- * Sort (67) + +- Exchange (66) + +- WindowGroupLimit (65) + +- * Sort (64) + +- Union (63) + :- * HashAggregate (22) + : +- Exchange (21) + : +- * HashAggregate (20) + : +- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.item (14) + :- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * HashAggregate (24) + : +- ReusedExchange (23) + :- * HashAggregate (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- * HashAggregate (29) + : +- ReusedExchange (28) + :- * HashAggregate (37) + : +- Exchange (36) + : +- * HashAggregate (35) + : +- * HashAggregate (34) + : +- ReusedExchange (33) + :- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * HashAggregate (39) + : +- ReusedExchange (38) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * HashAggregate (44) + : +- ReusedExchange (43) + :- * HashAggregate (52) + : +- Exchange (51) + : +- * HashAggregate (50) + : +- * HashAggregate (49) + : +- ReusedExchange (48) + :- * HashAggregate (57) + : +- Exchange (56) + : +- * HashAggregate (55) + : +- * HashAggregate (54) + : +- ReusedExchange (53) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * HashAggregate (59) + +- ReusedExchange (58) (1) Scan parquet spark_catalog.default.store_sales @@ -83,384 +84,391 @@ ReadSchema: struct + +(4) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) -(4) ReusedExchange [Reuses operator id: 75] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(5) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(6) CometBroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] +Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] +(8) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] + +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#12, s_store_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) - -(9) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#12] +(10) CometFilter +Input [2]: [s_store_sk#12, s_store_id#13] +Condition : isnotnull(s_store_sk#12) -(10) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [2]: [s_store_sk#12, s_store_id#13] +Arguments: [s_store_sk#12, s_store_id#13] -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] +Right output [2]: [s_store_sk#12, s_store_id#13] +Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight -(12) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#12] +(13) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#13] +Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13] -(13) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +(14) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(14) CometFilter -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Condition : isnotnull(i_item_sk#13) +(15) CometFilter +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Condition : isnotnull(i_item_sk#14) -(15) ColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +(16) CometBroadcastExchange +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -(16) BroadcastExchange -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13] +Right output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [ss_item_sk#1], [i_item_sk#14], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(18) CometProject +Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18] -(18) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +(19) ColumnarToRow [codegen id : 1] +Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18] -(19) HashAggregate [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +(20) HashAggregate [codegen id : 1] +Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] +Aggregate Attributes [2]: [sum#19, isEmpty#20] +Results [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#21, isEmpty#22] -(20) Exchange -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) Exchange +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#21, isEmpty#22] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +(22) HashAggregate [codegen id : 2] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#21, isEmpty#22] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [9]: [i_category#16 AS i_category#23, i_class#15 AS i_class#24, i_brand#14 AS i_brand#25, i_product_name#17 AS i_product_name#26, d_year#8 AS d_year#27, d_qoy#10 AS d_qoy#28, d_moy#9 AS d_moy#29, s_store_id#12 AS s_store_id#30, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#31] - -(22) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] - -(23) HashAggregate [codegen id : 10] -Input [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] -Keys [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39] -Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22] -Results [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22 AS sumsales#44] - -(24) HashAggregate [codegen id : 10] -Input [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sumsales#44] -Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] - -(25) Exchange -Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(26) HashAggregate [codegen id : 11] -Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] -Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#49] -Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] - -(27) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] - -(28) HashAggregate [codegen id : 16] -Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] -Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] -Functions [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22] -Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22 AS sumsales#64] - -(29) HashAggregate [codegen id : 16] -Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#64] -Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] -Functions [1]: [partial_sum(sumsales#64)] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] - -(30) Exchange -Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] -Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(31) HashAggregate [codegen id : 17] -Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] -Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] -Functions [1]: [sum(sumsales#64)] -Aggregate Attributes [1]: [sum(sumsales#64)#69] -Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#64)#69 AS sumsales#72] - -(32) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] - -(33) HashAggregate [codegen id : 22] -Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] -Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] -Functions [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22] -Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22 AS sumsales#85] - -(34) HashAggregate [codegen id : 22] -Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#85] -Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] -Functions [1]: [partial_sum(sumsales#85)] -Aggregate Attributes [2]: [sum#86, isEmpty#87] -Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] - -(35) Exchange -Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] -Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(36) HashAggregate [codegen id : 23] -Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] -Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] -Functions [1]: [sum(sumsales#85)] -Aggregate Attributes [1]: [sum(sumsales#85)#90] -Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#85)#90 AS sumsales#94] - -(37) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] - -(38) HashAggregate [codegen id : 28] -Input [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] -Keys [8]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102] -Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22] -Results [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22 AS sumsales#107] - -(39) HashAggregate [codegen id : 28] -Input [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sumsales#107] -Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] -Functions [1]: [partial_sum(sumsales#107)] -Aggregate Attributes [2]: [sum#108, isEmpty#109] -Results [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] - -(40) Exchange -Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] -Arguments: hashpartitioning(i_category#95, i_class#96, i_brand#97, i_product_name#98, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(41) HashAggregate [codegen id : 29] -Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] -Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] -Functions [1]: [sum(sumsales#107)] -Aggregate Attributes [1]: [sum(sumsales#107)#112] -Results [9]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#107)#112 AS sumsales#117] - -(42) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] - -(43) HashAggregate [codegen id : 34] -Input [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] -Keys [8]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125] -Functions [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22] -Results [4]: [i_category#118, i_class#119, i_brand#120, sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22 AS sumsales#130] - -(44) HashAggregate [codegen id : 34] -Input [4]: [i_category#118, i_class#119, i_brand#120, sumsales#130] -Keys [3]: [i_category#118, i_class#119, i_brand#120] -Functions [1]: [partial_sum(sumsales#130)] -Aggregate Attributes [2]: [sum#131, isEmpty#132] -Results [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] - -(45) Exchange -Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] -Arguments: hashpartitioning(i_category#118, i_class#119, i_brand#120, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(46) HashAggregate [codegen id : 35] -Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] -Keys [3]: [i_category#118, i_class#119, i_brand#120] -Functions [1]: [sum(sumsales#130)] -Aggregate Attributes [1]: [sum(sumsales#130)#135] -Results [9]: [i_category#118, i_class#119, i_brand#120, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#130)#135 AS sumsales#141] - -(47) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] - -(48) HashAggregate [codegen id : 40] -Input [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] -Keys [8]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149] -Functions [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22] -Results [3]: [i_category#142, i_class#143, sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22 AS sumsales#154] - -(49) HashAggregate [codegen id : 40] -Input [3]: [i_category#142, i_class#143, sumsales#154] -Keys [2]: [i_category#142, i_class#143] -Functions [1]: [partial_sum(sumsales#154)] -Aggregate Attributes [2]: [sum#155, isEmpty#156] -Results [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] - -(50) Exchange -Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] -Arguments: hashpartitioning(i_category#142, i_class#143, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(51) HashAggregate [codegen id : 41] -Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] -Keys [2]: [i_category#142, i_class#143] -Functions [1]: [sum(sumsales#154)] -Aggregate Attributes [1]: [sum(sumsales#154)#159] -Results [9]: [i_category#142, i_class#143, null AS i_brand#160, null AS i_product_name#161, null AS d_year#162, null AS d_qoy#163, null AS d_moy#164, null AS s_store_id#165, sum(sumsales#154)#159 AS sumsales#166] - -(52) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] - -(53) HashAggregate [codegen id : 46] -Input [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] -Keys [8]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174] -Functions [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22] -Results [2]: [i_category#167, sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22 AS sumsales#179] - -(54) HashAggregate [codegen id : 46] -Input [2]: [i_category#167, sumsales#179] -Keys [1]: [i_category#167] -Functions [1]: [partial_sum(sumsales#179)] -Aggregate Attributes [2]: [sum#180, isEmpty#181] -Results [3]: [i_category#167, sum#182, isEmpty#183] - -(55) Exchange -Input [3]: [i_category#167, sum#182, isEmpty#183] -Arguments: hashpartitioning(i_category#167, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(56) HashAggregate [codegen id : 47] -Input [3]: [i_category#167, sum#182, isEmpty#183] -Keys [1]: [i_category#167] -Functions [1]: [sum(sumsales#179)] -Aggregate Attributes [1]: [sum(sumsales#179)#184] -Results [9]: [i_category#167, null AS i_class#185, null AS i_brand#186, null AS i_product_name#187, null AS d_year#188, null AS d_qoy#189, null AS d_moy#190, null AS s_store_id#191, sum(sumsales#179)#184 AS sumsales#192] - -(57) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] - -(58) HashAggregate [codegen id : 52] -Input [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] -Keys [8]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200] -Functions [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22] -Results [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22 AS sumsales#205] - -(59) HashAggregate [codegen id : 52] -Input [1]: [sumsales#205] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#23] +Results [9]: [i_category#17 AS i_category#24, i_class#16 AS i_class#25, i_brand#15 AS i_brand#26, i_product_name#18 AS i_product_name#27, d_year#9 AS d_year#28, d_qoy#11 AS d_qoy#29, d_moy#10 AS d_moy#30, s_store_id#13 AS s_store_id#31, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#23 as decimal(38,2)) AS sumsales#32] + +(23) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, s_store_id#40, sum#41, isEmpty#42] + +(24) HashAggregate [codegen id : 4] +Input [10]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, s_store_id#40, sum#41, isEmpty#42] +Keys [8]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, s_store_id#40] +Functions [1]: [sum(coalesce((ss_sales_price#43 * cast(ss_quantity#44 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#43 * cast(ss_quantity#44 as decimal(10,0))), 0.00))#23] +Results [8]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, sum(coalesce((ss_sales_price#43 * cast(ss_quantity#44 as decimal(10,0))), 0.00))#23 AS sumsales#45] + +(25) HashAggregate [codegen id : 4] +Input [8]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, sumsales#45] +Keys [7]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39] +Functions [1]: [partial_sum(sumsales#45)] +Aggregate Attributes [2]: [sum#46, isEmpty#47] +Results [9]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, sum#48, isEmpty#49] + +(26) Exchange +Input [9]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, sum#48, isEmpty#49] +Arguments: hashpartitioning(i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(27) HashAggregate [codegen id : 5] +Input [9]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, sum#48, isEmpty#49] +Keys [7]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39] +Functions [1]: [sum(sumsales#45)] +Aggregate Attributes [1]: [sum(sumsales#45)#50] +Results [9]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, null AS s_store_id#51, sum(sumsales#45)#50 AS sumsales#52] + +(28) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58, d_moy#59, s_store_id#60, sum#61, isEmpty#62] + +(29) HashAggregate [codegen id : 7] +Input [10]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58, d_moy#59, s_store_id#60, sum#61, isEmpty#62] +Keys [8]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58, d_moy#59, s_store_id#60] +Functions [1]: [sum(coalesce((ss_sales_price#63 * cast(ss_quantity#64 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#63 * cast(ss_quantity#64 as decimal(10,0))), 0.00))#23] +Results [7]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58, sum(coalesce((ss_sales_price#63 * cast(ss_quantity#64 as decimal(10,0))), 0.00))#23 AS sumsales#65] + +(30) HashAggregate [codegen id : 7] +Input [7]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58, sumsales#65] +Keys [6]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58] +Functions [1]: [partial_sum(sumsales#65)] +Aggregate Attributes [2]: [sum#66, isEmpty#67] +Results [8]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58, sum#68, isEmpty#69] + +(31) Exchange +Input [8]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58, sum#68, isEmpty#69] +Arguments: hashpartitioning(i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(32) HashAggregate [codegen id : 8] +Input [8]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58, sum#68, isEmpty#69] +Keys [6]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58] +Functions [1]: [sum(sumsales#65)] +Aggregate Attributes [1]: [sum(sumsales#65)#70] +Results [9]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58, null AS d_moy#71, null AS s_store_id#72, sum(sumsales#65)#70 AS sumsales#73] + +(33) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sum#82, isEmpty#83] + +(34) HashAggregate [codegen id : 10] +Input [10]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sum#82, isEmpty#83] +Keys [8]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81] +Functions [1]: [sum(coalesce((ss_sales_price#84 * cast(ss_quantity#85 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#84 * cast(ss_quantity#85 as decimal(10,0))), 0.00))#23] +Results [6]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum(coalesce((ss_sales_price#84 * cast(ss_quantity#85 as decimal(10,0))), 0.00))#23 AS sumsales#86] + +(35) HashAggregate [codegen id : 10] +Input [6]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sumsales#86] +Keys [5]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78] +Functions [1]: [partial_sum(sumsales#86)] +Aggregate Attributes [2]: [sum#87, isEmpty#88] +Results [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] + +(36) Exchange +Input [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] +Arguments: hashpartitioning(i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(37) HashAggregate [codegen id : 11] +Input [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] +Keys [5]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78] +Functions [1]: [sum(sumsales#86)] +Aggregate Attributes [1]: [sum(sumsales#86)#91] +Results [9]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, null AS d_qoy#92, null AS d_moy#93, null AS s_store_id#94, sum(sumsales#86)#91 AS sumsales#95] + +(38) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103, sum#104, isEmpty#105] + +(39) HashAggregate [codegen id : 13] +Input [10]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103, sum#104, isEmpty#105] +Keys [8]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103] +Functions [1]: [sum(coalesce((ss_sales_price#106 * cast(ss_quantity#107 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#106 * cast(ss_quantity#107 as decimal(10,0))), 0.00))#23] +Results [5]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum(coalesce((ss_sales_price#106 * cast(ss_quantity#107 as decimal(10,0))), 0.00))#23 AS sumsales#108] + +(40) HashAggregate [codegen id : 13] +Input [5]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sumsales#108] +Keys [4]: [i_category#96, i_class#97, i_brand#98, i_product_name#99] +Functions [1]: [partial_sum(sumsales#108)] +Aggregate Attributes [2]: [sum#109, isEmpty#110] +Results [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] + +(41) Exchange +Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] +Arguments: hashpartitioning(i_category#96, i_class#97, i_brand#98, i_product_name#99, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(42) HashAggregate [codegen id : 14] +Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] +Keys [4]: [i_category#96, i_class#97, i_brand#98, i_product_name#99] +Functions [1]: [sum(sumsales#108)] +Aggregate Attributes [1]: [sum(sumsales#108)#113] +Results [9]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, null AS d_year#114, null AS d_qoy#115, null AS d_moy#116, null AS s_store_id#117, sum(sumsales#108)#113 AS sumsales#118] + +(43) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#119, i_class#120, i_brand#121, i_product_name#122, d_year#123, d_qoy#124, d_moy#125, s_store_id#126, sum#127, isEmpty#128] + +(44) HashAggregate [codegen id : 16] +Input [10]: [i_category#119, i_class#120, i_brand#121, i_product_name#122, d_year#123, d_qoy#124, d_moy#125, s_store_id#126, sum#127, isEmpty#128] +Keys [8]: [i_category#119, i_class#120, i_brand#121, i_product_name#122, d_year#123, d_qoy#124, d_moy#125, s_store_id#126] +Functions [1]: [sum(coalesce((ss_sales_price#129 * cast(ss_quantity#130 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#129 * cast(ss_quantity#130 as decimal(10,0))), 0.00))#23] +Results [4]: [i_category#119, i_class#120, i_brand#121, sum(coalesce((ss_sales_price#129 * cast(ss_quantity#130 as decimal(10,0))), 0.00))#23 AS sumsales#131] + +(45) HashAggregate [codegen id : 16] +Input [4]: [i_category#119, i_class#120, i_brand#121, sumsales#131] +Keys [3]: [i_category#119, i_class#120, i_brand#121] +Functions [1]: [partial_sum(sumsales#131)] +Aggregate Attributes [2]: [sum#132, isEmpty#133] +Results [5]: [i_category#119, i_class#120, i_brand#121, sum#134, isEmpty#135] + +(46) Exchange +Input [5]: [i_category#119, i_class#120, i_brand#121, sum#134, isEmpty#135] +Arguments: hashpartitioning(i_category#119, i_class#120, i_brand#121, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(47) HashAggregate [codegen id : 17] +Input [5]: [i_category#119, i_class#120, i_brand#121, sum#134, isEmpty#135] +Keys [3]: [i_category#119, i_class#120, i_brand#121] +Functions [1]: [sum(sumsales#131)] +Aggregate Attributes [1]: [sum(sumsales#131)#136] +Results [9]: [i_category#119, i_class#120, i_brand#121, null AS i_product_name#137, null AS d_year#138, null AS d_qoy#139, null AS d_moy#140, null AS s_store_id#141, sum(sumsales#131)#136 AS sumsales#142] + +(48) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#143, i_class#144, i_brand#145, i_product_name#146, d_year#147, d_qoy#148, d_moy#149, s_store_id#150, sum#151, isEmpty#152] + +(49) HashAggregate [codegen id : 19] +Input [10]: [i_category#143, i_class#144, i_brand#145, i_product_name#146, d_year#147, d_qoy#148, d_moy#149, s_store_id#150, sum#151, isEmpty#152] +Keys [8]: [i_category#143, i_class#144, i_brand#145, i_product_name#146, d_year#147, d_qoy#148, d_moy#149, s_store_id#150] +Functions [1]: [sum(coalesce((ss_sales_price#153 * cast(ss_quantity#154 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#153 * cast(ss_quantity#154 as decimal(10,0))), 0.00))#23] +Results [3]: [i_category#143, i_class#144, sum(coalesce((ss_sales_price#153 * cast(ss_quantity#154 as decimal(10,0))), 0.00))#23 AS sumsales#155] + +(50) HashAggregate [codegen id : 19] +Input [3]: [i_category#143, i_class#144, sumsales#155] +Keys [2]: [i_category#143, i_class#144] +Functions [1]: [partial_sum(sumsales#155)] +Aggregate Attributes [2]: [sum#156, isEmpty#157] +Results [4]: [i_category#143, i_class#144, sum#158, isEmpty#159] + +(51) Exchange +Input [4]: [i_category#143, i_class#144, sum#158, isEmpty#159] +Arguments: hashpartitioning(i_category#143, i_class#144, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(52) HashAggregate [codegen id : 20] +Input [4]: [i_category#143, i_class#144, sum#158, isEmpty#159] +Keys [2]: [i_category#143, i_class#144] +Functions [1]: [sum(sumsales#155)] +Aggregate Attributes [1]: [sum(sumsales#155)#160] +Results [9]: [i_category#143, i_class#144, null AS i_brand#161, null AS i_product_name#162, null AS d_year#163, null AS d_qoy#164, null AS d_moy#165, null AS s_store_id#166, sum(sumsales#155)#160 AS sumsales#167] + +(53) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#168, i_class#169, i_brand#170, i_product_name#171, d_year#172, d_qoy#173, d_moy#174, s_store_id#175, sum#176, isEmpty#177] + +(54) HashAggregate [codegen id : 22] +Input [10]: [i_category#168, i_class#169, i_brand#170, i_product_name#171, d_year#172, d_qoy#173, d_moy#174, s_store_id#175, sum#176, isEmpty#177] +Keys [8]: [i_category#168, i_class#169, i_brand#170, i_product_name#171, d_year#172, d_qoy#173, d_moy#174, s_store_id#175] +Functions [1]: [sum(coalesce((ss_sales_price#178 * cast(ss_quantity#179 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#178 * cast(ss_quantity#179 as decimal(10,0))), 0.00))#23] +Results [2]: [i_category#168, sum(coalesce((ss_sales_price#178 * cast(ss_quantity#179 as decimal(10,0))), 0.00))#23 AS sumsales#180] + +(55) HashAggregate [codegen id : 22] +Input [2]: [i_category#168, sumsales#180] +Keys [1]: [i_category#168] +Functions [1]: [partial_sum(sumsales#180)] +Aggregate Attributes [2]: [sum#181, isEmpty#182] +Results [3]: [i_category#168, sum#183, isEmpty#184] + +(56) Exchange +Input [3]: [i_category#168, sum#183, isEmpty#184] +Arguments: hashpartitioning(i_category#168, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(57) HashAggregate [codegen id : 23] +Input [3]: [i_category#168, sum#183, isEmpty#184] +Keys [1]: [i_category#168] +Functions [1]: [sum(sumsales#180)] +Aggregate Attributes [1]: [sum(sumsales#180)#185] +Results [9]: [i_category#168, null AS i_class#186, null AS i_brand#187, null AS i_product_name#188, null AS d_year#189, null AS d_qoy#190, null AS d_moy#191, null AS s_store_id#192, sum(sumsales#180)#185 AS sumsales#193] + +(58) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#194, i_class#195, i_brand#196, i_product_name#197, d_year#198, d_qoy#199, d_moy#200, s_store_id#201, sum#202, isEmpty#203] + +(59) HashAggregate [codegen id : 25] +Input [10]: [i_category#194, i_class#195, i_brand#196, i_product_name#197, d_year#198, d_qoy#199, d_moy#200, s_store_id#201, sum#202, isEmpty#203] +Keys [8]: [i_category#194, i_class#195, i_brand#196, i_product_name#197, d_year#198, d_qoy#199, d_moy#200, s_store_id#201] +Functions [1]: [sum(coalesce((ss_sales_price#204 * cast(ss_quantity#205 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#204 * cast(ss_quantity#205 as decimal(10,0))), 0.00))#23] +Results [1]: [sum(coalesce((ss_sales_price#204 * cast(ss_quantity#205 as decimal(10,0))), 0.00))#23 AS sumsales#206] + +(60) HashAggregate [codegen id : 25] +Input [1]: [sumsales#206] Keys: [] -Functions [1]: [partial_sum(sumsales#205)] -Aggregate Attributes [2]: [sum#206, isEmpty#207] -Results [2]: [sum#208, isEmpty#209] +Functions [1]: [partial_sum(sumsales#206)] +Aggregate Attributes [2]: [sum#207, isEmpty#208] +Results [2]: [sum#209, isEmpty#210] -(60) Exchange -Input [2]: [sum#208, isEmpty#209] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] +(61) Exchange +Input [2]: [sum#209, isEmpty#210] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(61) HashAggregate [codegen id : 53] -Input [2]: [sum#208, isEmpty#209] +(62) HashAggregate [codegen id : 26] +Input [2]: [sum#209, isEmpty#210] Keys: [] -Functions [1]: [sum(sumsales#205)] -Aggregate Attributes [1]: [sum(sumsales#205)#210] -Results [9]: [null AS i_category#211, null AS i_class#212, null AS i_brand#213, null AS i_product_name#214, null AS d_year#215, null AS d_qoy#216, null AS d_moy#217, null AS s_store_id#218, sum(sumsales#205)#210 AS sumsales#219] +Functions [1]: [sum(sumsales#206)] +Aggregate Attributes [1]: [sum(sumsales#206)#211] +Results [9]: [null AS i_category#212, null AS i_class#213, null AS i_brand#214, null AS i_product_name#215, null AS d_year#216, null AS d_qoy#217, null AS d_moy#218, null AS s_store_id#219, sum(sumsales#206)#211 AS sumsales#220] -(62) Union +(63) Union -(63) Sort [codegen id : 54] -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] -Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 +(64) Sort [codegen id : 27] +Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#32] +Arguments: [i_category#24 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0 -(64) WindowGroupLimit -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] -Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Partial +(65) WindowGroupLimit +Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#32] +Arguments: [i_category#24], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Partial -(65) Exchange -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] -Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(66) Exchange +Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#32] +Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(66) Sort [codegen id : 55] -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] -Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 +(67) Sort [codegen id : 28] +Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#32] +Arguments: [i_category#24 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0 -(67) WindowGroupLimit -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] -Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Final +(68) WindowGroupLimit +Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#32] +Arguments: [i_category#24], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Final -(68) Window -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] -Arguments: [rank(sumsales#31) windowspecdefinition(i_category#23, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#220], [i_category#23], [sumsales#31 DESC NULLS LAST] +(69) Window +Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#32] +Arguments: [rank(sumsales#32) windowspecdefinition(i_category#24, sumsales#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#221], [i_category#24], [sumsales#32 DESC NULLS LAST] -(69) Filter [codegen id : 56] -Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] -Condition : (rk#220 <= 100) +(70) Filter [codegen id : 29] +Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#32, rk#221] +Condition : (rk#221 <= 100) -(70) TakeOrderedAndProject -Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] -Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#220 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +(71) TakeOrderedAndProject +Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#32, rk#221] +Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#32 ASC NULLS FIRST, rk#221 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#32, rk#221] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(71) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] +(72) Scan parquet spark_catalog.default.date_dim +Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(72) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#221) AND (d_month_seq#221 >= 1212)) AND (d_month_seq#221 <= 1223)) AND isnotnull(d_date_sk#7)) +(73) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) -(73) CometProject -Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(74) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(74) ColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(75) ColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(75) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(76) BroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt index 4d2d3972d9..f68ac8dfc2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt @@ -1,32 +1,32 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (56) + WholeStageCodegen (29) Filter [rk] InputAdapter Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (55) + WholeStageCodegen (28) Sort [i_category,sumsales] InputAdapter Exchange [i_category] #1 WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (54) + WholeStageCodegen (27) Sort [i_category,sumsales] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -37,90 +37,84 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) + CometBroadcastExchange #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange #5 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - WholeStageCodegen (10) + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 + WholeStageCodegen (4) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - WholeStageCodegen (16) + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 + WholeStageCodegen (7) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) + WholeStageCodegen (11) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - WholeStageCodegen (22) + Exchange [i_category,i_class,i_brand,i_product_name,d_year] #9 + WholeStageCodegen (10) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) + WholeStageCodegen (14) HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name] #9 - WholeStageCodegen (28) + Exchange [i_category,i_class,i_brand,i_product_name] #10 + WholeStageCodegen (13) HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) + WholeStageCodegen (17) HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class,i_brand] #10 - WholeStageCodegen (34) + Exchange [i_category,i_class,i_brand] #11 + WholeStageCodegen (16) HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) + WholeStageCodegen (20) HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class] #11 - WholeStageCodegen (40) + Exchange [i_category,i_class] #12 + WholeStageCodegen (19) HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) + WholeStageCodegen (23) HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category] #12 - WholeStageCodegen (46) + Exchange [i_category] #13 + WholeStageCodegen (22) HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) + WholeStageCodegen (26) HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange #13 - WholeStageCodegen (52) + Exchange #14 + WholeStageCodegen (25) HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt index 6a81a2ec1c..90e9a457ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt @@ -1,60 +1,62 @@ == Physical Plan == -TakeOrderedAndProject (56) -+- * Project (55) - +- Window (54) - +- * Sort (53) - +- Exchange (52) - +- * HashAggregate (51) - +- Exchange (50) - +- * HashAggregate (49) - +- Union (48) - :- * HashAggregate (37) - : +- Exchange (36) - : +- * HashAggregate (35) - : +- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (32) - : +- * BroadcastHashJoin LeftSemi BuildRight (31) - : :- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (30) - : +- * Project (29) - : +- * Filter (28) - : +- Window (27) - : +- WindowGroupLimit (26) - : +- * Sort (25) - : +- * HashAggregate (24) - : +- Exchange (23) - : +- * HashAggregate (22) - : +- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * ColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.store_sales (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- ReusedExchange (19) - :- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * HashAggregate (39) - : +- ReusedExchange (38) - +- * HashAggregate (47) - +- Exchange (46) - +- * HashAggregate (45) - +- * HashAggregate (44) - +- ReusedExchange (43) +TakeOrderedAndProject (58) ++- * Project (57) + +- Window (56) + +- * Sort (55) + +- Exchange (54) + +- * HashAggregate (53) + +- Exchange (52) + +- * HashAggregate (51) + +- Union (50) + :- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * ColumnarToRow (9) + : : +- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- BroadcastExchange (34) + : +- * BroadcastHashJoin LeftSemi BuildRight (33) + : :- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store (10) + : +- BroadcastExchange (32) + : +- * Project (31) + : +- * Filter (30) + : +- Window (29) + : +- WindowGroupLimit (28) + : +- * Sort (27) + : +- * HashAggregate (26) + : +- Exchange (25) + : +- * HashAggregate (24) + : +- * ColumnarToRow (23) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store_sales (13) + : : +- CometBroadcastExchange (17) + : : +- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.store (15) + : +- ReusedExchange (20) + :- * HashAggregate (44) + : +- Exchange (43) + : +- * HashAggregate (42) + : +- * HashAggregate (41) + : +- ReusedExchange (40) + +- * HashAggregate (49) + +- Exchange (48) + +- * HashAggregate (47) + +- * HashAggregate (46) + +- ReusedExchange (45) (1) Scan parquet spark_catalog.default.store_sales @@ -69,300 +71,310 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 61] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] +(7) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] + +(9) ColumnarToRow [codegen id : 5] +Input [2]: [ss_store_sk#1, ss_net_profit#2] -(7) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] +(10) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) +(11) CometFilter +Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Condition : isnotnull(s_store_sk#7) -(9) ColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] +(12) ColumnarToRow [codegen id : 4] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(13) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) CometFilter -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(12) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(14) CometFilter +Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_store_sk#10) -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] +(15) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : isnotnull(s_store_sk#13) +(16) CometFilter +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) -(15) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#13, s_state#14] +(17) CometBroadcastExchange +Input [2]: [s_store_sk#14, s_state#15] +Arguments: [s_store_sk#14, s_state#15] -(16) BroadcastExchange -Input [2]: [s_store_sk#13, s_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(18) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Right output [2]: [s_store_sk#14, s_state#15] +Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None +(19) CometProject +Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#15] +Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15], [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] -(18) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] -(19) ReusedExchange [Reuses operator id: 61] -Output [1]: [d_date_sk#15] +(21) CometBroadcastHashJoin +Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] +Right output [1]: [d_date_sk#16] +Arguments: [ss_sold_date_sk#12], [d_date_sk#16], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(22) CometProject +Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15, d_date_sk#16] +Arguments: [ss_net_profit#11, s_state#15], [ss_net_profit#11, s_state#15] + +(23) ColumnarToRow [codegen id : 1] +Input [2]: [ss_net_profit#11, s_state#15] + +(24) HashAggregate [codegen id : 1] +Input [2]: [ss_net_profit#11, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] +Aggregate Attributes [1]: [sum#17] +Results [2]: [s_state#15, sum#18] + +(25) Exchange +Input [2]: [s_state#15, sum#18] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(26) HashAggregate [codegen id : 2] +Input [2]: [s_state#15, sum#18] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#19] +Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#19,17,2) AS _w0#20, s_state#15] + +(27) Sort [codegen id : 2] +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [s_state#15 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 -(21) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#14] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] - -(22) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#14] -Keys [1]: [s_state#14] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [s_state#14, sum#17] - -(23) Exchange -Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(24) HashAggregate [codegen id : 5] -Input [2]: [s_state#14, sum#17] -Keys [1]: [s_state#14] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] -Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] - -(25) Sort [codegen id : 5] -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 - -(26) WindowGroupLimit -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(27) Window -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] - -(28) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] -Condition : (ranking#20 <= 5) - -(29) Project [codegen id : 6] -Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] - -(30) BroadcastExchange -Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(31) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [s_state#8] -Right keys [1]: [s_state#14] +(28) WindowGroupLimit +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [s_state#15], [_w0#20 DESC NULLS LAST], rank(_w0#20), 5, Final + +(29) Window +Input [3]: [s_state#15, _w0#20, s_state#15] +Arguments: [rank(_w0#20) windowspecdefinition(s_state#15, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#15], [_w0#20 DESC NULLS LAST] + +(30) Filter [codegen id : 3] +Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] +Condition : (ranking#21 <= 5) + +(31) Project [codegen id : 3] +Output [1]: [s_state#15] +Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] + +(32) BroadcastExchange +Input [1]: [s_state#15] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] + +(33) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [s_state#9] +Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(32) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(34) BroadcastExchange +Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] +Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(34) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_county#7, s_state#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] +(36) Project [codegen id : 5] +Output [3]: [ss_net_profit#2, s_county#8, s_state#9] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#9] -(35) HashAggregate [codegen id : 8] -Input [3]: [ss_net_profit#2, s_county#7, s_state#8] -Keys [2]: [s_state#8, s_county#7] +(37) HashAggregate [codegen id : 5] +Input [3]: [ss_net_profit#2, s_county#8, s_state#9] +Keys [2]: [s_state#9, s_county#8] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [s_state#8, s_county#7, sum#22] +Aggregate Attributes [1]: [sum#22] +Results [3]: [s_state#9, s_county#8, sum#23] -(36) Exchange -Input [3]: [s_state#8, s_county#7, sum#22] -Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(38) Exchange +Input [3]: [s_state#9, s_county#8, sum#23] +Arguments: hashpartitioning(s_state#9, s_county#8, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(37) HashAggregate [codegen id : 9] -Input [3]: [s_state#8, s_county#7, sum#22] -Keys [2]: [s_state#8, s_county#7] +(39) HashAggregate [codegen id : 6] +Input [3]: [s_state#9, s_county#8, sum#23] +Keys [2]: [s_state#9, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) as decimal(27,2)) AS total_sum#24, s_state#8, s_county#7, 0 AS g_state#25, 0 AS g_county#26, 0 AS lochierarchy#27] - -(38) ReusedExchange [Reuses operator id: 36] -Output [3]: [s_state#28, s_county#29, sum#30] - -(39) HashAggregate [codegen id : 18] -Input [3]: [s_state#28, s_county#29, sum#30] -Keys [2]: [s_state#28, s_county#29] -Functions [1]: [sum(UnscaledValue(ss_net_profit#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#31))#23] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#31))#23,17,2) AS total_sum#32, s_state#28] - -(40) HashAggregate [codegen id : 18] -Input [2]: [total_sum#32, s_state#28] -Keys [1]: [s_state#28] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [s_state#28, sum#35, isEmpty#36] - -(41) Exchange -Input [3]: [s_state#28, sum#35, isEmpty#36] -Arguments: hashpartitioning(s_state#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(42) HashAggregate [codegen id : 19] -Input [3]: [s_state#28, sum#35, isEmpty#36] -Keys [1]: [s_state#28] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#37] -Results [6]: [sum(total_sum#32)#37 AS total_sum#38, s_state#28, null AS s_county#39, 0 AS g_state#40, 1 AS g_county#41, 1 AS lochierarchy#42] - -(43) ReusedExchange [Reuses operator id: 36] -Output [3]: [s_state#43, s_county#44, sum#45] - -(44) HashAggregate [codegen id : 28] -Input [3]: [s_state#43, s_county#44, sum#45] -Keys [2]: [s_state#43, s_county#44] -Functions [1]: [sum(UnscaledValue(ss_net_profit#46))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#46))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#46))#23,17,2) AS total_sum#47] - -(45) HashAggregate [codegen id : 28] -Input [1]: [total_sum#47] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#9, s_county#8, 0 AS g_state#26, 0 AS g_county#27, 0 AS lochierarchy#28] + +(40) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_state#29, s_county#30, sum#31] + +(41) HashAggregate [codegen id : 12] +Input [3]: [s_state#29, s_county#30, sum#31] +Keys [2]: [s_state#29, s_county#30] +Functions [1]: [sum(UnscaledValue(ss_net_profit#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#32))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#32))#24,17,2) AS total_sum#33, s_state#29] + +(42) HashAggregate [codegen id : 12] +Input [2]: [total_sum#33, s_state#29] +Keys [1]: [s_state#29] +Functions [1]: [partial_sum(total_sum#33)] +Aggregate Attributes [2]: [sum#34, isEmpty#35] +Results [3]: [s_state#29, sum#36, isEmpty#37] + +(43) Exchange +Input [3]: [s_state#29, sum#36, isEmpty#37] +Arguments: hashpartitioning(s_state#29, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(44) HashAggregate [codegen id : 13] +Input [3]: [s_state#29, sum#36, isEmpty#37] +Keys [1]: [s_state#29] +Functions [1]: [sum(total_sum#33)] +Aggregate Attributes [1]: [sum(total_sum#33)#38] +Results [6]: [sum(total_sum#33)#38 AS total_sum#39, s_state#29, null AS s_county#40, 0 AS g_state#41, 1 AS g_county#42, 1 AS lochierarchy#43] + +(45) ReusedExchange [Reuses operator id: 38] +Output [3]: [s_state#44, s_county#45, sum#46] + +(46) HashAggregate [codegen id : 19] +Input [3]: [s_state#44, s_county#45, sum#46] +Keys [2]: [s_state#44, s_county#45] +Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#48] + +(47) HashAggregate [codegen id : 19] +Input [1]: [total_sum#48] Keys: [] -Functions [1]: [partial_sum(total_sum#47)] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [2]: [sum#50, isEmpty#51] +Functions [1]: [partial_sum(total_sum#48)] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [2]: [sum#51, isEmpty#52] -(46) Exchange -Input [2]: [sum#50, isEmpty#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(48) Exchange +Input [2]: [sum#51, isEmpty#52] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(47) HashAggregate [codegen id : 29] -Input [2]: [sum#50, isEmpty#51] +(49) HashAggregate [codegen id : 20] +Input [2]: [sum#51, isEmpty#52] Keys: [] -Functions [1]: [sum(total_sum#47)] -Aggregate Attributes [1]: [sum(total_sum#47)#52] -Results [6]: [sum(total_sum#47)#52 AS total_sum#53, null AS s_state#54, null AS s_county#55, 1 AS g_state#56, 1 AS g_county#57, 2 AS lochierarchy#58] +Functions [1]: [sum(total_sum#48)] +Aggregate Attributes [1]: [sum(total_sum#48)#53] +Results [6]: [sum(total_sum#48)#53 AS total_sum#54, null AS s_state#55, null AS s_county#56, 1 AS g_state#57, 1 AS g_county#58, 2 AS lochierarchy#59] -(48) Union +(50) Union -(49) HashAggregate [codegen id : 30] -Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +(51) HashAggregate [codegen id : 21] +Input [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] +Keys [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +Results [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] -(50) Exchange -Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -Arguments: hashpartitioning(total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(52) Exchange +Input [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] +Arguments: hashpartitioning(total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(51) HashAggregate [codegen id : 31] -Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +(53) HashAggregate [codegen id : 22] +Input [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] +Keys [6]: [total_sum#25, s_state#9, s_county#8, g_state#26, g_county#27, lochierarchy#28] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, CASE WHEN (g_county#26 = 0) THEN s_state#8 END AS _w0#59] +Results [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, CASE WHEN (g_county#27 = 0) THEN s_state#9 END AS _w0#60] -(52) Exchange -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#59] -Arguments: hashpartitioning(lochierarchy#27, _w0#59, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(54) Exchange +Input [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, _w0#60] +Arguments: hashpartitioning(lochierarchy#28, _w0#60, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(53) Sort [codegen id : 32] -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#59] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#24 DESC NULLS LAST], false, 0 +(55) Sort [codegen id : 23] +Input [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, _w0#60] +Arguments: [lochierarchy#28 ASC NULLS FIRST, _w0#60 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST], false, 0 -(54) Window -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#59] -Arguments: [rank(total_sum#24) windowspecdefinition(lochierarchy#27, _w0#59, total_sum#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#27, _w0#59], [total_sum#24 DESC NULLS LAST] +(56) Window +Input [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, _w0#60] +Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#28, _w0#60, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#28, _w0#60], [total_sum#25 DESC NULLS LAST] -(55) Project [codegen id : 33] -Output [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#60] -Input [6]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#59, rank_within_parent#60] +(57) Project [codegen id : 24] +Output [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, rank_within_parent#61] +Input [6]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, _w0#60, rank_within_parent#61] -(56) TakeOrderedAndProject -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#60] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#60] +(58) TakeOrderedAndProject +Input [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, rank_within_parent#61] +Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN s_state#9 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#25, s_state#9, s_county#8, lochierarchy#28, rank_within_parent#61] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (61) -+- * ColumnarToRow (60) - +- CometProject (59) - +- CometFilter (58) - +- CometScan parquet spark_catalog.default.date_dim (57) +BroadcastExchange (63) ++- * ColumnarToRow (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.date_dim (59) -(57) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#61] +(59) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(58) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#61] -Condition : (((isnotnull(d_month_seq#61) AND (d_month_seq#61 >= 1212)) AND (d_month_seq#61 <= 1223)) AND isnotnull(d_date_sk#5)) +(60) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(59) CometProject -Input [2]: [d_date_sk#5, d_month_seq#61] +(61) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(60) ColumnarToRow [codegen id : 1] +(62) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(61) BroadcastExchange +(63) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt index 2fd04badf8..a7c639a495 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt @@ -1,32 +1,32 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (33) + WholeStageCodegen (24) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (32) + WholeStageCodegen (23) Sort [lochierarchy,_w0,total_sum] InputAdapter Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (31) + WholeStageCodegen (22) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] [_w0] InputAdapter Exchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (30) + WholeStageCodegen (21) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union - WholeStageCodegen (9) + WholeStageCodegen (6) HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] InputAdapter Exchange [s_state,s_county] #3 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] Project [ss_net_profit,s_county,s_state] BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -37,63 +37,60 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (4) BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (3) Project [s_state] Filter [ranking] InputAdapter Window [_w0,s_state] WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) + WholeStageCodegen (2) Sort [s_state,_w0] HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] InputAdapter - Exchange [s_state] #7 - WholeStageCodegen (4) + Exchange [s_state] #8 + WholeStageCodegen (1) HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) + CometBroadcastExchange #9 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (13) HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter - Exchange [s_state] #9 - WholeStageCodegen (18) + Exchange [s_state] #10 + WholeStageCodegen (12) HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) + WholeStageCodegen (20) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter - Exchange #10 - WholeStageCodegen (28) + Exchange #11 + WholeStageCodegen (19) HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt index c885738383..8bff19a727 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt @@ -1,74 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * SortMergeJoin LeftOuter (65) - :- * Sort (58) - : +- Exchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * ColumnarToRow (3) - : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * ColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * ColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * ColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * ColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * ColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * ColumnarToRow (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.promotion (51) - +- * Sort (64) - +- Exchange (63) - +- * ColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.catalog_returns (59) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- * Project (63) + +- * SortMergeJoin LeftOuter (62) + :- * Sort (55) + : +- Exchange (54) + : +- * Project (53) + : +- * BroadcastHashJoin LeftOuter BuildRight (52) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * ColumnarToRow (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (35) + : : : : +- CometBroadcastHashJoin (34) + : : : : :- CometProject (29) + : : : : : +- CometBroadcastHashJoin (28) + : : : : : :- CometProject (23) + : : : : : : +- CometBroadcastHashJoin (22) + : : : : : : :- CometProject (17) + : : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : : :- CometProject (12) + : : : : : : : : +- CometBroadcastHashJoin (11) + : : : : : : : : :- CometProject (7) + : : : : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : : : : :- CometFilter (2) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- CometBroadcastExchange (5) + : : : : : : : : : +- CometFilter (4) + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (3) + : : : : : : : : +- CometBroadcastExchange (10) + : : : : : : : : +- CometFilter (9) + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (8) + : : : : : : : +- CometBroadcastExchange (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : : +- CometBroadcastExchange (21) + : : : : : : +- CometProject (20) + : : : : : : +- CometFilter (19) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) + : : : : : +- CometBroadcastExchange (27) + : : : : : +- CometProject (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan parquet spark_catalog.default.household_demographics (24) + : : : : +- CometBroadcastExchange (33) + : : : : +- CometProject (32) + : : : : +- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.date_dim (30) + : : : +- CometBroadcastExchange (38) + : : : +- CometFilter (37) + : : : +- CometScan parquet spark_catalog.default.date_dim (36) + : : +- BroadcastExchange (45) + : : +- * ColumnarToRow (44) + : : +- CometFilter (43) + : : +- CometScan parquet spark_catalog.default.date_dim (42) + : +- BroadcastExchange (51) + : +- * ColumnarToRow (50) + : +- CometFilter (49) + : +- CometScan parquet spark_catalog.default.promotion (48) + +- * Sort (61) + +- Exchange (60) + +- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.catalog_returns (56) (1) Scan parquet spark_catalog.default.catalog_sales @@ -83,10 +80,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -(7) BroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) +(6) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] +Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +(7) CometProject Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -(10) Scan parquet spark_catalog.default.warehouse +(8) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [w_warehouse_sk#14, w_warehouse_name#15] -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +(12) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#16, i_item_desc#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#16, i_item_desc#17] - -(19) BroadcastExchange +(15) CometBroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#16, i_item_desc#17] -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +Right output [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(17) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(22) Scan parquet spark_catalog.default.customer_demographics +(18) Scan parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#18, cd_marital_status#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_marital_status), EqualTo(cd_marital_status,M), IsNotNull(cd_demo_sk)] ReadSchema: struct -(23) CometFilter +(19) CometFilter Input [2]: [cd_demo_sk#18, cd_marital_status#19] Condition : ((isnotnull(cd_marital_status#19) AND (cd_marital_status#19 = M)) AND isnotnull(cd_demo_sk#18)) -(24) CometProject +(20) CometProject Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) ColumnarToRow [codegen id : 4] +(21) CometBroadcastExchange Input [1]: [cd_demo_sk#18] +Arguments: [cd_demo_sk#18] -(26) BroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [cd_demo_sk#18] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(23) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] +Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(29) Scan parquet spark_catalog.default.household_demographics +(24) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#20, hd_buy_potential#21] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct -(30) CometFilter +(25) CometFilter Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Condition : ((isnotnull(hd_buy_potential#21) AND (hd_buy_potential#21 = 1001-5000 )) AND isnotnull(hd_demo_sk#20)) -(31) CometProject +(26) CometProject Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) ColumnarToRow [codegen id : 5] +(27) CometBroadcastExchange Input [1]: [hd_demo_sk#20] +Arguments: [hd_demo_sk#20] -(33) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(28) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [hd_demo_sk#20] +Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(29) CometProject Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +(30) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +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 -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(31) CometFilter +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)) -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +(32) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] + +(33) CometBroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] + +(34) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight + +(35) CometProject Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(39) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] +(36) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_week_seq#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) +(37) CometFilter +Input [2]: [d_date_sk#26, d_week_seq#27] +Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) -(41) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] +(38) CometBroadcastExchange +Input [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_date_sk#26, d_week_seq#27] -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] +(39) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +Right output [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None +(40) CometProject +Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] +(41) ColumnarToRow [codegen id : 3] +Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(45) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] +(42) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_date#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) +(43) CometFilter +Input [2]: [d_date_sk#28, d_date#29] +Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) -(47) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] +(44) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_date#29] -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(45) BroadcastExchange +Input [2]: [d_date_sk#28, d_date#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(49) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#28] Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) +Join condition: (d_date#29 > date_add(d_date#23, 5)) -(50) Project [codegen id : 10] +(47) Project [codegen id : 3] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] +Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -(51) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] +(48) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) +(49) CometFilter +Input [1]: [p_promo_sk#30] +Condition : isnotnull(p_promo_sk#30) -(53) ColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] +(50) ColumnarToRow [codegen id : 2] +Input [1]: [p_promo_sk#30] -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(51) BroadcastExchange +Input [1]: [p_promo_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(55) BroadcastHashJoin [codegen id : 10] +(52) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] +Right keys [1]: [p_promo_sk#30] Join type: LeftOuter Join condition: None -(56) Project [codegen id : 10] +(53) Project [codegen id : 3] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -(57) Exchange +(54) Exchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(58) Sort [codegen id : 11] +(55) Sort [codegen id : 4] Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 -(59) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +(56) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) +(57) CometFilter +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] +(58) CometProject +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] -(62) ColumnarToRow [codegen id : 12] -Input [2]: [cr_item_sk#30, cr_order_number#31] +(59) ColumnarToRow [codegen id : 5] +Input [2]: [cr_item_sk#31, cr_order_number#32] -(63) Exchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(60) Exchange +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(64) Sort [codegen id : 13] -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST], false, 0 +(61) Sort [codegen id : 6] +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 14] +(62) SortMergeJoin [codegen id : 7] Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#30, cr_order_number#31] +Right keys [2]: [cr_item_sk#31, cr_order_number#32] Join type: LeftOuter Join condition: None -(66) Project [codegen id : 14] +(63) Project [codegen id : 7] Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -(67) HashAggregate [codegen id : 14] +(64) HashAggregate [codegen id : 7] Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Aggregate Attributes [1]: [count#34] +Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] -(68) Exchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(65) Exchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(69) HashAggregate [codegen id : 15] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +(66) HashAggregate [codegen id : 8] +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#35 AS no_promo#36, count(1)#35 AS promo#37, count(1)#35 AS total_cnt#38] +Aggregate Attributes [1]: [count(1)#36] +Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#36 AS no_promo#37, count(1)#36 AS promo#38, count(1)#36 AS total_cnt#39] -(70) TakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] -Arguments: 100, [total_cnt#38 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] +(67) TakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#37, promo#38, total_cnt#39] +Arguments: 100, [total_cnt#39 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#37, promo#38, total_cnt#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(71) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +(68) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] 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 -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] -Condition : ((((isnotnull(d_year#39) AND (d_year#39 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(69) CometFilter +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)) -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +(70) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(75) BroadcastExchange +(72) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt index 17fc9dee7e..bea1fd4a16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt @@ -1,38 +1,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - WholeStageCodegen (15) + WholeStageCodegen (8) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - WholeStageCodegen (14) + WholeStageCodegen (7) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] Project [w_warehouse_name,i_item_desc,d_week_seq] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (4) Sort [cs_item_sk,cs_order_number] InputAdapter Exchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) + WholeStageCodegen (3) 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] BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -43,72 +43,50 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_year,d_date_sk,d_week_seq,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [d_week_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange #4 + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange #7 + CometProject [cd_demo_sk] + CometFilter [cd_marital_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange #8 + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange #10 + CometFilter [d_week_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) + BroadcastExchange #11 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) + BroadcastExchange #12 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (6) Sort [cr_item_sk,cr_order_number] InputAdapter - Exchange [cr_item_sk,cr_order_number] #12 - WholeStageCodegen (12) + Exchange [cr_item_sk,cr_order_number] #13 + WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt index ad8c33ca1d..0ae3bf650a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt @@ -2,74 +2,74 @@ TakeOrderedAndProject (71) +- * Project (70) +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * ColumnarToRow (36) - : : : +- CometFilter (35) - : : : +- CometScan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (17) + : : : +- * HashAggregate (16) + : : : +- Exchange (15) + : : : +- * HashAggregate (14) + : : : +- * ColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- BroadcastExchange (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * HashAggregate (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- CometBroadcastExchange (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- BroadcastExchange (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * HashAggregate (47) + : +- * ColumnarToRow (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometProject (42) + : : +- CometBroadcastHashJoin (41) + : : :- CometFilter (37) + : : : +- CometScan parquet spark_catalog.default.customer (36) + : : +- CometBroadcastExchange (40) + : : +- CometFilter (39) + : : +- CometScan parquet spark_catalog.default.web_sales (38) + : +- ReusedExchange (43) +- BroadcastExchange (68) +- * HashAggregate (67) +- Exchange (66) +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * ColumnarToRow (55) - : : +- CometFilter (54) - : : +- CometScan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) + +- * ColumnarToRow (64) + +- CometProject (63) + +- CometBroadcastHashJoin (62) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (61) (1) Scan parquet spark_catalog.default.customer @@ -83,10 +83,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] Condition : isnotnull(ss_customer_sk#5) -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Right output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: [c_customer_sk#1], [ss_customer_sk#5], Inner, BuildRight -(9) Project [codegen id : 3] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +(7) CometProject Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 75] +(8) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#9, d_year#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#9, d_year#10] +Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [2]: [d_date_sk#9, d_year#10] +Arguments: [d_date_sk#9, d_year#10] -(12) Project [codegen id : 3] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] +(11) CometBroadcastHashJoin +Left output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#9, d_year#10] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight + +(12) CometProject Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] + +(13) ColumnarToRow [codegen id : 1] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] -(13) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] Aggregate Attributes [1]: [sum#11] Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -(14) Exchange +(15) Exchange Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 16] +(16) HashAggregate [codegen id : 8] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] -(16) Filter [codegen id : 16] +(17) Filter [codegen id : 8] Input [2]: [customer_id#14, year_total#15] Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) -(17) Scan parquet spark_catalog.default.customer +(18) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter +(19) CometFilter Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) -(19) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] - (20) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Batched: true @@ -176,79 +180,86 @@ ReadSchema: struct Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) -(22) ColumnarToRow [codegen id : 4] +(22) CometBroadcastExchange Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Arguments: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -(23) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +Right output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Arguments: [c_customer_sk#16], [ss_customer_sk#20], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#16] -Right keys [1]: [ss_customer_sk#20] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] +(24) CometProject Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Arguments: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22], [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] -(26) ReusedExchange [Reuses operator id: 79] +(25) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(26) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) + +(27) CometBroadcastExchange +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24, d_year#25] -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] +(28) CometBroadcastHashJoin +Left output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] +Right output [2]: [d_date_sk#24, d_year#25] +Arguments: [ss_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight + +(29) CometProject Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] +Arguments: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25], [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] + +(30) ColumnarToRow [codegen id : 2] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -(29) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] Aggregate Attributes [1]: [sum#26] Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -(30) Exchange +(32) Exchange Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(31) HashAggregate [codegen id : 7] +(33) HashAggregate [codegen id : 3] Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] -(32) BroadcastExchange +(34) BroadcastExchange Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 16] +(35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#28] Join type: Inner Join condition: None -(34) Scan parquet spark_catalog.default.customer +(36) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter +(37) CometFilter Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) -(36) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] - -(37) Scan parquet spark_catalog.default.web_sales +(38) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] @@ -256,90 +267,85 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(38) CometFilter +(39) CometFilter Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] Condition : isnotnull(ws_bill_customer_sk#36) -(39) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] - -(40) BroadcastExchange +(40) CometBroadcastExchange Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#32] -Right keys [1]: [ws_bill_customer_sk#36] -Join type: Inner -Join condition: None +(41) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Right output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Arguments: [c_customer_sk#32], [ws_bill_customer_sk#36], Inner, BuildRight -(42) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] +(42) CometProject Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38], [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] -(43) ReusedExchange [Reuses operator id: 75] +(43) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#40, d_year#41] -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None +(44) CometBroadcastHashJoin +Left output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] +Right output [2]: [d_date_sk#40, d_year#41] +Arguments: [ws_sold_date_sk#38], [d_date_sk#40], Inner, BuildRight -(45) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] +(45) CometProject Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41], [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -(46) HashAggregate [codegen id : 10] +(46) ColumnarToRow [codegen id : 4] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] + +(47) HashAggregate [codegen id : 4] Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] Aggregate Attributes [1]: [sum#42] Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -(47) Exchange +(48) Exchange Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(48) HashAggregate [codegen id : 11] +(49) HashAggregate [codegen id : 5] Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] -(49) Filter [codegen id : 11] +(50) Filter [codegen id : 5] Input [2]: [customer_id#45, year_total#46] Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) -(50) BroadcastExchange +(51) BroadcastExchange Input [2]: [customer_id#45, year_total#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(51) BroadcastHashJoin [codegen id : 16] +(52) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#45] Join type: Inner Join condition: None -(52) Project [codegen id : 16] +(53) Project [codegen id : 8] Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] -(53) Scan parquet spark_catalog.default.customer +(54) Scan parquet spark_catalog.default.customer Output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter +(55) CometFilter Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) -(55) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] - (56) Scan parquet spark_catalog.default.web_sales Output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Batched: true @@ -352,37 +358,35 @@ ReadSchema: struct Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] Condition : isnotnull(ws_bill_customer_sk#51) -(58) ColumnarToRow [codegen id : 12] +(58) CometBroadcastExchange Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Arguments: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -(59) BroadcastExchange -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(59) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +Right output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Arguments: [c_customer_sk#47], [ws_bill_customer_sk#51], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#47] -Right keys [1]: [ws_bill_customer_sk#51] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] +(60) CometProject Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Arguments: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53], [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] -(62) ReusedExchange [Reuses operator id: 79] +(61) ReusedExchange [Reuses operator id: 27] Output [2]: [d_date_sk#55, d_year#56] -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#53] -Right keys [1]: [d_date_sk#55] -Join type: Inner -Join condition: None +(62) CometBroadcastHashJoin +Left output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] +Right output [2]: [d_date_sk#55, d_year#56] +Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight -(64) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] +(63) CometProject Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] +Arguments: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56], [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -(65) HashAggregate [codegen id : 14] +(64) ColumnarToRow [codegen id : 6] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] + +(65) HashAggregate [codegen id : 6] Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] @@ -391,9 +395,9 @@ Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum# (66) Exchange Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(67) HashAggregate [codegen id : 15] +(67) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] @@ -402,15 +406,15 @@ Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue( (68) BroadcastExchange Input [2]: [customer_id#59, year_total#60] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(69) BroadcastHashJoin [codegen id : 16] +(69) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#59] Join type: Inner Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) -(70) Project [codegen id : 16] +(70) Project [codegen id : 8] Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] @@ -420,7 +424,7 @@ Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULL ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (75) +- * ColumnarToRow (74) +- CometFilter (73) @@ -443,7 +447,7 @@ Input [2]: [d_date_sk#9, d_year#10] (75) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (79) @@ -468,9 +472,9 @@ Input [2]: [d_date_sk#24, d_year#25] (79) BroadcastExchange Input [2]: [d_date_sk#24, d_year#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt index 26989b0c00..a2cb15b4e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - WholeStageCodegen (16) + WholeStageCodegen (8) Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] @@ -9,114 +9,98 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + WholeStageCodegen (2) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange #7 CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #13 + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + WholeStageCodegen (6) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt index 5c8480a96a..d45346b6ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt @@ -1,133 +1,136 @@ == Physical Plan == -TakeOrderedAndProject (129) -+- * Project (128) - +- * SortMergeJoin Inner (127) - :- * Sort (71) - : +- Exchange (70) - : +- * Filter (69) - : +- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- Union (62) - : :- * Project (23) - : : +- * SortMergeJoin LeftOuter (22) - : : :- * Sort (15) - : : : +- Exchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- * Sort (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : :- * Project (42) - : : +- * SortMergeJoin LeftOuter (41) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : :- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * ColumnarToRow (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) - : : : : +- ReusedExchange (27) - : : : +- ReusedExchange (30) - : : +- * Sort (40) - : : +- Exchange (39) - : : +- * ColumnarToRow (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometScan parquet spark_catalog.default.store_returns (35) - : +- * Project (61) - : +- * SortMergeJoin LeftOuter (60) - : :- * Sort (53) - : : +- Exchange (52) - : : +- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * ColumnarToRow (45) - : : : : +- CometFilter (44) - : : : : +- CometScan parquet spark_catalog.default.web_sales (43) - : : : +- ReusedExchange (46) - : : +- ReusedExchange (49) - : +- * Sort (59) - : +- Exchange (58) - : +- * ColumnarToRow (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan parquet spark_catalog.default.web_returns (54) - +- * Sort (126) - +- Exchange (125) - +- * Filter (124) - +- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- * HashAggregate (120) - +- Exchange (119) - +- * HashAggregate (118) - +- Union (117) - :- * Project (86) - : +- * SortMergeJoin LeftOuter (85) - : :- * Sort (82) - : : +- Exchange (81) - : : +- * Project (80) - : : +- * BroadcastHashJoin Inner BuildRight (79) - : : :- * Project (77) - : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : :- * ColumnarToRow (74) - : : : : +- CometFilter (73) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) - : : : +- ReusedExchange (75) - : : +- ReusedExchange (78) - : +- * Sort (84) - : +- ReusedExchange (83) - :- * Project (101) - : +- * SortMergeJoin LeftOuter (100) - : :- * Sort (97) - : : +- Exchange (96) - : : +- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * ColumnarToRow (89) - : : : : +- CometFilter (88) - : : : : +- CometScan parquet spark_catalog.default.store_sales (87) - : : : +- ReusedExchange (90) - : : +- ReusedExchange (93) - : +- * Sort (99) - : +- ReusedExchange (98) - +- * Project (116) - +- * SortMergeJoin LeftOuter (115) - :- * Sort (112) - : +- Exchange (111) - : +- * Project (110) - : +- * BroadcastHashJoin Inner BuildRight (109) - : :- * Project (107) - : : +- * BroadcastHashJoin Inner BuildRight (106) - : : :- * ColumnarToRow (104) - : : : +- CometFilter (103) - : : : +- CometScan parquet spark_catalog.default.web_sales (102) - : : +- ReusedExchange (105) - : +- ReusedExchange (108) - +- * Sort (114) - +- ReusedExchange (113) +TakeOrderedAndProject (132) ++- * Project (131) + +- * SortMergeJoin Inner (130) + :- * Sort (72) + : +- Exchange (71) + : +- * Filter (70) + : +- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- * HashAggregate (66) + : +- Exchange (65) + : +- * HashAggregate (64) + : +- Union (63) + : :- * Project (24) + : : +- * SortMergeJoin LeftOuter (23) + : : :- * Sort (16) + : : : +- Exchange (15) + : : : +- * ColumnarToRow (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.date_dim (9) + : : +- * Sort (22) + : : +- Exchange (21) + : : +- * ColumnarToRow (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : :- * Project (43) + : : +- * SortMergeJoin LeftOuter (42) + : : :- * Sort (35) + : : : +- Exchange (34) + : : : +- * ColumnarToRow (33) + : : : +- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometFilter (26) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (25) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- * Sort (41) + : : +- Exchange (40) + : : +- * ColumnarToRow (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.store_returns (36) + : +- * Project (62) + : +- * SortMergeJoin LeftOuter (61) + : :- * Sort (54) + : : +- Exchange (53) + : : +- * ColumnarToRow (52) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometProject (48) + : : : +- CometBroadcastHashJoin (47) + : : : :- CometFilter (45) + : : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- * Sort (60) + : +- Exchange (59) + : +- * ColumnarToRow (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.web_returns (55) + +- * Sort (129) + +- Exchange (128) + +- * Filter (127) + +- * HashAggregate (126) + +- Exchange (125) + +- * HashAggregate (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- Union (120) + :- * Project (89) + : +- * SortMergeJoin LeftOuter (88) + : :- * Sort (85) + : : +- Exchange (84) + : : +- * ColumnarToRow (83) + : : +- CometProject (82) + : : +- CometBroadcastHashJoin (81) + : : :- CometProject (77) + : : : +- CometBroadcastHashJoin (76) + : : : :- CometFilter (74) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) + : : : +- ReusedExchange (75) + : : +- CometBroadcastExchange (80) + : : +- CometFilter (79) + : : +- CometScan parquet spark_catalog.default.date_dim (78) + : +- * Sort (87) + : +- ReusedExchange (86) + :- * Project (104) + : +- * SortMergeJoin LeftOuter (103) + : :- * Sort (100) + : : +- Exchange (99) + : : +- * ColumnarToRow (98) + : : +- CometProject (97) + : : +- CometBroadcastHashJoin (96) + : : :- CometProject (94) + : : : +- CometBroadcastHashJoin (93) + : : : :- CometFilter (91) + : : : : +- CometScan parquet spark_catalog.default.store_sales (90) + : : : +- ReusedExchange (92) + : : +- ReusedExchange (95) + : +- * Sort (102) + : +- ReusedExchange (101) + +- * Project (119) + +- * SortMergeJoin LeftOuter (118) + :- * Sort (115) + : +- Exchange (114) + : +- * ColumnarToRow (113) + : +- CometProject (112) + : +- CometBroadcastHashJoin (111) + : :- CometProject (109) + : : +- CometBroadcastHashJoin (108) + : : :- CometFilter (106) + : : : +- CometScan parquet spark_catalog.default.web_sales (105) + : : +- ReusedExchange (107) + : +- ReusedExchange (110) + +- * Sort (117) + +- ReusedExchange (116) (1) Scan parquet spark_catalog.default.catalog_sales @@ -142,99 +145,106 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) ColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(8) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +(8) CometProject Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 133] +(9) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +(11) CometBroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: [d_date_sk#13, d_year#14] + +(12) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Right output [2]: [d_date_sk#13, d_year#14] +Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight + +(13) CometProject Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] + +(14) ColumnarToRow [codegen id : 1] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -(14) Exchange +(15) Exchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) Sort [codegen id : 4] +(16) Sort [codegen id : 2] Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 -(16) Scan parquet spark_catalog.default.catalog_returns +(17) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(17) CometFilter +(18) CometFilter Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) -(18) CometProject +(19) CometProject Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(19) ColumnarToRow [codegen id : 5] +(20) ColumnarToRow [codegen id : 3] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(20) Exchange +(21) Exchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) Sort [codegen id : 6] +(22) Sort [codegen id : 4] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] Arguments: [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 7] +(23) SortMergeJoin [codegen id : 5] Left keys [2]: [cs_order_number#2, cs_item_sk#1] Right keys [2]: [cr_order_number#16, cr_item_sk#15] Join type: LeftOuter Join condition: None -(23) Project [codegen id : 7] +(24) Project [codegen id : 5] Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(24) Scan parquet spark_catalog.default.store_sales +(25) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] @@ -242,84 +252,82 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(25) CometFilter +(26) CometFilter Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#22) -(26) ColumnarToRow [codegen id : 10] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(27) ReusedExchange [Reuses operator id: 8] +(27) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(28) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#22] -Right keys [1]: [i_item_sk#28] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight -(29) Project [codegen id : 10] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +(29) CometProject Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(30) ReusedExchange [Reuses operator id: 133] +(30) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#33, d_year#34] -(31) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#33] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Right output [2]: [d_date_sk#33, d_year#34] +Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight -(32) Project [codegen id : 10] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +(32) CometProject Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] + +(33) ColumnarToRow [codegen id : 6] +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -(33) Exchange +(34) Exchange Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(34) Sort [codegen id : 11] +(35) Sort [codegen id : 7] Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Arguments: [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST], false, 0 -(35) Scan parquet spark_catalog.default.store_returns +(36) Scan parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(36) CometFilter +(37) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(37) CometProject +(38) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(38) ColumnarToRow [codegen id : 12] +(39) ColumnarToRow [codegen id : 8] Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(39) Exchange +(40) Exchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(40) Sort [codegen id : 13] +(41) Sort [codegen id : 9] Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST], false, 0 -(41) SortMergeJoin [codegen id : 14] +(42) SortMergeJoin [codegen id : 10] Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] Join type: LeftOuter Join condition: None -(42) Project [codegen id : 14] +(43) Project [codegen id : 10] Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(43) Scan parquet spark_catalog.default.web_sales +(44) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] @@ -327,134 +335,132 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(44) CometFilter +(45) CometFilter Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Condition : isnotnull(ws_item_sk#42) -(45) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] - -(46) ReusedExchange [Reuses operator id: 8] +(46) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(47) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#42] -Right keys [1]: [i_item_sk#48] -Join type: Inner -Join condition: None +(47) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight -(48) Project [codegen id : 17] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +(48) CometProject Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(49) ReusedExchange [Reuses operator id: 133] +(49) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#46] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(50) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight -(51) Project [codegen id : 17] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +(51) CometProject Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(52) Exchange +(52) ColumnarToRow [codegen id : 11] Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(53) Sort [codegen id : 18] +(53) Exchange +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(54) Sort [codegen id : 12] Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Arguments: [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST], false, 0 -(54) Scan parquet spark_catalog.default.web_returns +(55) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(55) CometFilter +(56) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(56) CometProject +(57) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(57) ColumnarToRow [codegen id : 19] +(58) ColumnarToRow [codegen id : 13] Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(58) Exchange +(59) Exchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(59) Sort [codegen id : 20] +(60) Sort [codegen id : 14] Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST], false, 0 -(60) SortMergeJoin [codegen id : 21] +(61) SortMergeJoin [codegen id : 15] Left keys [2]: [ws_order_number#43, ws_item_sk#42] Right keys [2]: [wr_order_number#56, wr_item_sk#55] Join type: LeftOuter Join condition: None -(61) Project [codegen id : 21] +(62) Project [codegen id : 15] Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(62) Union +(63) Union -(63) HashAggregate [codegen id : 22] +(64) HashAggregate [codegen id : 16] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(64) Exchange +(65) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(65) HashAggregate [codegen id : 23] +(66) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(66) HashAggregate [codegen id : 23] +(67) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum#62, sum#63] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -(67) Exchange +(68) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(68) HashAggregate [codegen id : 24] +(69) HashAggregate [codegen id : 18] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] -(69) Filter [codegen id : 24] +(70) Filter [codegen id : 18] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] Condition : isnotnull(sales_cnt#68) -(70) Exchange +(71) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(71) Sort [codegen id : 25] +(72) Sort [codegen id : 19] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 -(72) Scan parquet spark_catalog.default.catalog_sales +(73) Scan parquet spark_catalog.default.catalog_sales Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] @@ -462,65 +468,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(73) CometFilter +(74) CometFilter Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Condition : isnotnull(cs_item_sk#70) -(74) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] - -(75) ReusedExchange [Reuses operator id: 8] +(75) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(76) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#70] -Right keys [1]: [i_item_sk#76] -Join type: Inner -Join condition: None +(76) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Right output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70], [i_item_sk#76], Inner, BuildRight -(77) Project [codegen id : 28] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(77) CometProject Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(78) ReusedExchange [Reuses operator id: 137] +(78) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#81, d_year#82] +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 -(79) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#74] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None +(79) CometFilter +Input [2]: [d_date_sk#81, d_year#82] +Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(80) Project [codegen id : 28] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +(80) CometBroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] +Arguments: [d_date_sk#81, d_year#82] + +(81) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Right output [2]: [d_date_sk#81, d_year#82] +Arguments: [cs_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight + +(82) CometProject Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] + +(83) ColumnarToRow [codegen id : 20] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -(81) Exchange +(84) Exchange Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(82) Sort [codegen id : 29] +(85) Sort [codegen id : 21] Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 -(83) ReusedExchange [Reuses operator id: 20] +(86) ReusedExchange [Reuses operator id: 21] Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(84) Sort [codegen id : 31] +(87) Sort [codegen id : 23] Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] Arguments: [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST], false, 0 -(85) SortMergeJoin [codegen id : 32] +(88) SortMergeJoin [codegen id : 24] Left keys [2]: [cs_order_number#71, cs_item_sk#70] Right keys [2]: [cr_order_number#84, cr_item_sk#83] Join type: LeftOuter Join condition: None -(86) Project [codegen id : 32] +(89) Project [codegen id : 24] Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#87, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#88] Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(87) Scan parquet spark_catalog.default.store_sales +(90) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] Batched: true Location: InMemoryFileIndex [] @@ -528,65 +544,63 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#93), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(88) CometFilter +(91) CometFilter Input [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] Condition : isnotnull(ss_item_sk#89) -(89) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] - -(90) ReusedExchange [Reuses operator id: 8] +(92) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] -(91) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#89] -Right keys [1]: [i_item_sk#95] -Join type: Inner -Join condition: None +(93) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] +Right output [5]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] +Arguments: [ss_item_sk#89], [i_item_sk#95], Inner, BuildRight -(92) Project [codegen id : 35] -Output [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] +(94) CometProject Input [10]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] +Arguments: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99], [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] -(93) ReusedExchange [Reuses operator id: 137] +(95) ReusedExchange [Reuses operator id: 80] Output [2]: [d_date_sk#100, d_year#101] -(94) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#93] -Right keys [1]: [d_date_sk#100] -Join type: Inner -Join condition: None +(96) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] +Right output [2]: [d_date_sk#100, d_year#101] +Arguments: [ss_sold_date_sk#93], [d_date_sk#100], Inner, BuildRight -(95) Project [codegen id : 35] -Output [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] +(97) CometProject Input [11]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_date_sk#100, d_year#101] +Arguments: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101], [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] -(96) Exchange +(98) ColumnarToRow [codegen id : 25] Input [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] -Arguments: hashpartitioning(ss_ticket_number#90, ss_item_sk#89, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(97) Sort [codegen id : 36] +(99) Exchange +Input [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] +Arguments: hashpartitioning(ss_ticket_number#90, ss_item_sk#89, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(100) Sort [codegen id : 26] Input [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] Arguments: [ss_ticket_number#90 ASC NULLS FIRST, ss_item_sk#89 ASC NULLS FIRST], false, 0 -(98) ReusedExchange [Reuses operator id: 39] +(101) ReusedExchange [Reuses operator id: 40] Output [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] -(99) Sort [codegen id : 38] +(102) Sort [codegen id : 28] Input [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] Arguments: [sr_ticket_number#103 ASC NULLS FIRST, sr_item_sk#102 ASC NULLS FIRST], false, 0 -(100) SortMergeJoin [codegen id : 39] +(103) SortMergeJoin [codegen id : 29] Left keys [2]: [ss_ticket_number#90, ss_item_sk#89] Right keys [2]: [sr_ticket_number#103, sr_item_sk#102] Join type: LeftOuter Join condition: None -(101) Project [codegen id : 39] +(104) Project [codegen id : 29] Output [7]: [d_year#101, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, (ss_quantity#91 - coalesce(sr_return_quantity#104, 0)) AS sales_cnt#106, (ss_ext_sales_price#92 - coalesce(sr_return_amt#105, 0.00)) AS sales_amt#107] Input [13]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101, sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] -(102) Scan parquet spark_catalog.default.web_sales +(105) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] Batched: true Location: InMemoryFileIndex [] @@ -594,186 +608,184 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(103) CometFilter +(106) CometFilter Input [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] Condition : isnotnull(ws_item_sk#108) -(104) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] - -(105) ReusedExchange [Reuses operator id: 8] +(107) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] -(106) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#108] -Right keys [1]: [i_item_sk#114] -Join type: Inner -Join condition: None +(108) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] +Right output [5]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] +Arguments: [ws_item_sk#108], [i_item_sk#114], Inner, BuildRight -(107) Project [codegen id : 42] -Output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] +(109) CometProject Input [10]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] +Arguments: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118], [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] -(108) ReusedExchange [Reuses operator id: 137] +(110) ReusedExchange [Reuses operator id: 80] Output [2]: [d_date_sk#119, d_year#120] -(109) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#112] -Right keys [1]: [d_date_sk#119] -Join type: Inner -Join condition: None +(111) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] +Right output [2]: [d_date_sk#119, d_year#120] +Arguments: [ws_sold_date_sk#112], [d_date_sk#119], Inner, BuildRight -(110) Project [codegen id : 42] -Output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] +(112) CometProject Input [11]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_date_sk#119, d_year#120] +Arguments: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120], [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] + +(113) ColumnarToRow [codegen id : 30] +Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] -(111) Exchange +(114) Exchange Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] -Arguments: hashpartitioning(ws_order_number#109, ws_item_sk#108, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: hashpartitioning(ws_order_number#109, ws_item_sk#108, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(112) Sort [codegen id : 43] +(115) Sort [codegen id : 31] Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] Arguments: [ws_order_number#109 ASC NULLS FIRST, ws_item_sk#108 ASC NULLS FIRST], false, 0 -(113) ReusedExchange [Reuses operator id: 58] +(116) ReusedExchange [Reuses operator id: 59] Output [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] -(114) Sort [codegen id : 45] +(117) Sort [codegen id : 33] Input [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] Arguments: [wr_order_number#122 ASC NULLS FIRST, wr_item_sk#121 ASC NULLS FIRST], false, 0 -(115) SortMergeJoin [codegen id : 46] +(118) SortMergeJoin [codegen id : 34] Left keys [2]: [ws_order_number#109, ws_item_sk#108] Right keys [2]: [wr_order_number#122, wr_item_sk#121] Join type: LeftOuter Join condition: None -(116) Project [codegen id : 46] +(119) Project [codegen id : 34] Output [7]: [d_year#120, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, (ws_quantity#110 - coalesce(wr_return_quantity#123, 0)) AS sales_cnt#125, (ws_ext_sales_price#111 - coalesce(wr_return_amt#124, 0.00)) AS sales_amt#126] Input [13]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120, wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] -(117) Union +(120) Union -(118) HashAggregate [codegen id : 47] +(121) HashAggregate [codegen id : 35] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] -(119) Exchange +(122) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(120) HashAggregate [codegen id : 48] +(123) HashAggregate [codegen id : 36] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] -(121) HashAggregate [codegen id : 48] +(124) HashAggregate [codegen id : 36] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [partial_sum(sales_cnt#87), partial_sum(UnscaledValue(sales_amt#88))] Aggregate Attributes [2]: [sum#127, sum#128] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] -(122) Exchange +(125) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(123) HashAggregate [codegen id : 49] +(126) HashAggregate [codegen id : 37] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [sum(sales_cnt#87), sum(UnscaledValue(sales_amt#88))] Aggregate Attributes [2]: [sum(sales_cnt#87)#66, sum(UnscaledValue(sales_amt#88))#67] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#87)#66 AS sales_cnt#131, MakeDecimal(sum(UnscaledValue(sales_amt#88))#67,18,2) AS sales_amt#132] -(124) Filter [codegen id : 49] +(127) Filter [codegen id : 37] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] Condition : isnotnull(sales_cnt#131) -(125) Exchange +(128) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] -Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(126) Sort [codegen id : 50] +(129) Sort [codegen id : 38] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 -(127) SortMergeJoin [codegen id : 51] +(130) SortMergeJoin [codegen id : 39] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Join type: Inner Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#131 as decimal(17,2))) < 0.90000000000000000000) -(128) Project [codegen id : 51] +(131) Project [codegen id : 39] Output [10]: [d_year#82 AS prev_year#133, d_year#14 AS year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#131 AS prev_yr_cnt#135, sales_cnt#68 AS curr_yr_cnt#136, (sales_cnt#68 - sales_cnt#131) AS sales_cnt_diff#137, (sales_amt#69 - sales_amt#132) AS sales_amt_diff#138] Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] -(129) TakeOrderedAndProject +(132) TakeOrderedAndProject Input [10]: [prev_year#133, year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#135, curr_yr_cnt#136, sales_cnt_diff#137, sales_amt_diff#138] Arguments: 100, [sales_cnt_diff#137 ASC NULLS FIRST, sales_amt_diff#138 ASC NULLS FIRST], [prev_year#133, year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#135, curr_yr_cnt#136, sales_cnt_diff#137, sales_amt_diff#138] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (133) -+- * ColumnarToRow (132) - +- CometFilter (131) - +- CometScan parquet spark_catalog.default.date_dim (130) +BroadcastExchange (136) ++- * ColumnarToRow (135) + +- CometFilter (134) + +- CometScan parquet spark_catalog.default.date_dim (133) -(130) Scan parquet spark_catalog.default.date_dim +(133) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(131) CometFilter +(134) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(132) ColumnarToRow [codegen id : 1] +(135) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(133) BroadcastExchange +(136) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 -BroadcastExchange (137) -+- * ColumnarToRow (136) - +- CometFilter (135) - +- CometScan parquet spark_catalog.default.date_dim (134) +Subquery:4 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (140) ++- * ColumnarToRow (139) + +- CometFilter (138) + +- CometScan parquet spark_catalog.default.date_dim (137) -(134) Scan parquet spark_catalog.default.date_dim +(137) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#81, d_year#82] 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 -(135) CometFilter +(138) CometFilter Input [2]: [d_date_sk#81, d_year#82] Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(136) ColumnarToRow [codegen id : 1] +(139) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] -(137) BroadcastExchange +(140) BroadcastExchange Input [2]: [d_date_sk#81, d_year#82] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#75 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt index 9939b2fe21..fb78d64b13 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt @@ -1,41 +1,41 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt] - WholeStageCodegen (51) + WholeStageCodegen (39) Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter - WholeStageCodegen (25) + WholeStageCodegen (19) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (24) + WholeStageCodegen (18) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - WholeStageCodegen (23) + WholeStageCodegen (17) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - WholeStageCodegen (22) + WholeStageCodegen (16) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (7) + WholeStageCodegen (5) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (2) Sort [cs_order_number,cs_item_sk] InputAdapter Exchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -45,196 +45,185 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometBroadcastExchange #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometBroadcastExchange #7 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (4) Sort [cr_order_number,cr_item_sk] InputAdapter - Exchange [cr_order_number,cr_item_sk] #7 - WholeStageCodegen (5) + Exchange [cr_order_number,cr_item_sk] #8 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - WholeStageCodegen (14) + WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (7) Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #9 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (9) Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #9 - WholeStageCodegen (12) + Exchange [sr_ticket_number,sr_item_sk] #10 + WholeStageCodegen (8) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - WholeStageCodegen (21) + WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (18) + WholeStageCodegen (12) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (17) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ws_order_number,ws_item_sk] #11 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter - WholeStageCodegen (20) + WholeStageCodegen (14) Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_order_number,wr_item_sk] #11 - WholeStageCodegen (19) + Exchange [wr_order_number,wr_item_sk] #12 + WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter - WholeStageCodegen (50) + WholeStageCodegen (38) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - WholeStageCodegen (49) + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + WholeStageCodegen (37) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (48) + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + WholeStageCodegen (36) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - WholeStageCodegen (47) + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + WholeStageCodegen (35) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (32) + WholeStageCodegen (24) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (29) + WholeStageCodegen (21) Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (28) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [cs_order_number,cs_item_sk] #16 + WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 + BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange #18 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (31) + WholeStageCodegen (23) Sort [cr_order_number,cr_item_sk] InputAdapter - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - WholeStageCodegen (39) + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + WholeStageCodegen (29) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (36) + WholeStageCodegen (26) Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (35) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #19 + WholeStageCodegen (25) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter - WholeStageCodegen (38) + WholeStageCodegen (28) Sort [sr_ticket_number,sr_item_sk] InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - WholeStageCodegen (46) + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + WholeStageCodegen (34) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (43) + WholeStageCodegen (31) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (42) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ws_order_number,ws_item_sk] #20 + WholeStageCodegen (30) + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter - WholeStageCodegen (45) + WholeStageCodegen (33) Sort [wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt index 5c18652670..b72c8bebb6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt @@ -1,102 +1,103 @@ == Physical Plan == -TakeOrderedAndProject (98) -+- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- Union (94) - :- * HashAggregate (83) - : +- Exchange (82) - : +- * HashAggregate (81) - : +- Union (80) - : :- * Project (30) - : : +- * BroadcastHashJoin LeftOuter BuildRight (29) - : : :- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.store (7) - : : +- BroadcastExchange (28) - : : +- * HashAggregate (27) - : : +- Exchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * ColumnarToRow (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.store_returns (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (49) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (48) - : : :- BroadcastExchange (39) - : : : +- * HashAggregate (38) - : : : +- Exchange (37) - : : : +- * HashAggregate (36) - : : : +- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (32) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : : +- ReusedExchange (33) - : : +- * HashAggregate (47) - : : +- Exchange (46) - : : +- * HashAggregate (45) - : : +- * Project (44) - : : +- * BroadcastHashJoin Inner BuildRight (43) - : : :- * ColumnarToRow (41) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (40) - : : +- ReusedExchange (42) - : +- * Project (79) - : +- * BroadcastHashJoin LeftOuter BuildRight (78) - : :- * HashAggregate (64) - : : +- Exchange (63) - : : +- * HashAggregate (62) - : : +- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * ColumnarToRow (52) - : : : : +- CometFilter (51) - : : : : +- CometScan parquet spark_catalog.default.web_sales (50) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.web_page (56) - : +- BroadcastExchange (77) - : +- * HashAggregate (76) - : +- Exchange (75) - : +- * HashAggregate (74) - : +- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * ColumnarToRow (67) - : : : +- CometFilter (66) - : : : +- CometScan parquet spark_catalog.default.web_returns (65) - : : +- ReusedExchange (68) - : +- ReusedExchange (71) - :- * HashAggregate (88) - : +- Exchange (87) - : +- * HashAggregate (86) - : +- * HashAggregate (85) - : +- ReusedExchange (84) - +- * HashAggregate (93) - +- Exchange (92) - +- * HashAggregate (91) - +- * HashAggregate (90) - +- ReusedExchange (89) +TakeOrderedAndProject (99) ++- * HashAggregate (98) + +- Exchange (97) + +- * HashAggregate (96) + +- Union (95) + :- * HashAggregate (84) + : +- Exchange (83) + : +- * HashAggregate (82) + : +- Union (81) + : :- * Project (32) + : : +- * BroadcastHashJoin LeftOuter BuildRight (31) + : : :- * HashAggregate (17) + : : : +- Exchange (16) + : : : +- * HashAggregate (15) + : : : +- * ColumnarToRow (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.store (9) + : : +- BroadcastExchange (30) + : : +- * HashAggregate (29) + : : +- Exchange (28) + : : +- * HashAggregate (27) + : : +- * ColumnarToRow (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.store_returns (18) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : :- * Project (51) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (50) + : : :- BroadcastExchange (41) + : : : +- * HashAggregate (40) + : : : +- Exchange (39) + : : : +- * HashAggregate (38) + : : : +- * ColumnarToRow (37) + : : : +- CometProject (36) + : : : +- CometBroadcastHashJoin (35) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (33) + : : : +- ReusedExchange (34) + : : +- * HashAggregate (49) + : : +- Exchange (48) + : : +- * HashAggregate (47) + : : +- * ColumnarToRow (46) + : : +- CometProject (45) + : : +- CometBroadcastHashJoin (44) + : : :- CometScan parquet spark_catalog.default.catalog_returns (42) + : : +- ReusedExchange (43) + : +- * Project (80) + : +- * BroadcastHashJoin LeftOuter BuildRight (79) + : :- * HashAggregate (65) + : : +- Exchange (64) + : : +- * HashAggregate (63) + : : +- * ColumnarToRow (62) + : : +- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (56) + : : : +- CometBroadcastHashJoin (55) + : : : :- CometFilter (53) + : : : : +- CometScan parquet spark_catalog.default.web_sales (52) + : : : +- ReusedExchange (54) + : : +- CometBroadcastExchange (59) + : : +- CometFilter (58) + : : +- CometScan parquet spark_catalog.default.web_page (57) + : +- BroadcastExchange (78) + : +- * HashAggregate (77) + : +- Exchange (76) + : +- * HashAggregate (75) + : +- * ColumnarToRow (74) + : +- CometProject (73) + : +- CometBroadcastHashJoin (72) + : :- CometProject (70) + : : +- CometBroadcastHashJoin (69) + : : :- CometFilter (67) + : : : +- CometScan parquet spark_catalog.default.web_returns (66) + : : +- ReusedExchange (68) + : +- ReusedExchange (71) + :- * HashAggregate (89) + : +- Exchange (88) + : +- * HashAggregate (87) + : +- * HashAggregate (86) + : +- ReusedExchange (85) + +- * HashAggregate (94) + +- Exchange (93) + +- * HashAggregate (92) + +- * HashAggregate (91) + +- ReusedExchange (90) (1) Scan parquet spark_catalog.default.store_sales @@ -111,522 +112,522 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_date#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] + +(7) CometBroadcastHashJoin +Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +(8) CometProject Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -(7) Scan parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] +(9) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) +(10) CometFilter +Input [1]: [s_store_sk#8] +Condition : isnotnull(s_store_sk#8) -(9) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] +(11) CometBroadcastExchange +Input [1]: [s_store_sk#8] +Arguments: [s_store_sk#8] -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +Right output [1]: [s_store_sk#8] +Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None +(13) CometProject +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +(14) ColumnarToRow [codegen id : 1] +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] +(15) HashAggregate [codegen id : 1] +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Keys [1]: [s_store_sk#8] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] +Aggregate Attributes [2]: [sum#9, sum#10] +Results [3]: [s_store_sk#8, sum#11, sum#12] -(14) Exchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) Exchange +Input [3]: [s_store_sk#8, sum#11, sum#12] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] +(17) HashAggregate [codegen id : 4] +Input [3]: [s_store_sk#8, sum#11, sum#12] +Keys [1]: [s_store_sk#8] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#13, sum(UnscaledValue(ss_net_profit#3))#14] +Results [3]: [s_store_sk#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#13,17,2) AS sales#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#14,17,2) AS profit#16] -(16) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +(18) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, sr_returned_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(sr_returned_date_sk#20), dynamicpruningexpression(sr_returned_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(17) CometFilter -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) - -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] - -(19) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#21] - -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] - -(22) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#22] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] - -(25) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Keys [1]: [s_store_sk#22] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#23, sum#24] -Results [3]: [s_store_sk#22, sum#25, sum#26] - -(26) Exchange -Input [3]: [s_store_sk#22, sum#25, sum#26] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#22, sum#25, sum#26] -Keys [1]: [s_store_sk#22] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] -Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] - -(28) BroadcastExchange -Input [3]: [s_store_sk#22, returns#29, profit_loss#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#22] +(19) CometFilter +Input [4]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, sr_returned_date_sk#20] +Condition : isnotnull(sr_store_sk#17) + +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#22] + +(21) CometBroadcastHashJoin +Left output [4]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, sr_returned_date_sk#20] +Right output [1]: [d_date_sk#22] +Arguments: [sr_returned_date_sk#20], [d_date_sk#22], Inner, BuildRight + +(22) CometProject +Input [5]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, sr_returned_date_sk#20, d_date_sk#22] +Arguments: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19], [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19] + +(23) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#23] + +(24) CometBroadcastHashJoin +Left output [3]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19] +Right output [1]: [s_store_sk#23] +Arguments: [sr_store_sk#17], [s_store_sk#23], Inner, BuildRight + +(25) CometProject +Input [4]: [sr_store_sk#17, sr_return_amt#18, sr_net_loss#19, s_store_sk#23] +Arguments: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23], [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] + +(26) ColumnarToRow [codegen id : 2] +Input [3]: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] + +(27) HashAggregate [codegen id : 2] +Input [3]: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] +Keys [1]: [s_store_sk#23] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#18)), partial_sum(UnscaledValue(sr_net_loss#19))] +Aggregate Attributes [2]: [sum#24, sum#25] +Results [3]: [s_store_sk#23, sum#26, sum#27] + +(28) Exchange +Input [3]: [s_store_sk#23, sum#26, sum#27] +Arguments: hashpartitioning(s_store_sk#23, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(29) HashAggregate [codegen id : 3] +Input [3]: [s_store_sk#23, sum#26, sum#27] +Keys [1]: [s_store_sk#23] +Functions [2]: [sum(UnscaledValue(sr_return_amt#18)), sum(UnscaledValue(sr_net_loss#19))] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#18))#28, sum(UnscaledValue(sr_net_loss#19))#29] +Results [3]: [s_store_sk#23, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#28,17,2) AS returns#30, MakeDecimal(sum(UnscaledValue(sr_net_loss#19))#29,17,2) AS profit_loss#31] + +(30) BroadcastExchange +Input [3]: [s_store_sk#23, returns#30, profit_loss#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(31) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [s_store_sk#8] +Right keys [1]: [s_store_sk#23] Join type: LeftOuter Join condition: None -(30) Project [codegen id : 8] -Output [5]: [store channel AS channel#31, s_store_sk#7 AS id#32, sales#14, coalesce(returns#29, 0.00) AS returns#33, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#34] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] +(32) Project [codegen id : 4] +Output [5]: [store channel AS channel#32, s_store_sk#8 AS id#33, sales#15, coalesce(returns#30, 0.00) AS returns#34, (profit#16 - coalesce(profit_loss#31, 0.00)) AS profit#35] +Input [6]: [s_store_sk#8, sales#15, profit#16, s_store_sk#23, returns#30, profit_loss#31] -(31) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(33) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38, cs_sold_date_sk#39] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(cs_sold_date_sk#39), dynamicpruningexpression(cs_sold_date_sk#39 IN dynamicpruning#40)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] - -(33) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#40] - -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] - -(36) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum#41, sum#42] -Results [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(37) Exchange -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(38) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] -Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] - -(39) BroadcastExchange -Input [3]: [cs_call_center_sk#35, sales#47, profit#48] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(40) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(34) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#41] + +(35) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38, cs_sold_date_sk#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#39], [d_date_sk#41], Inner, BuildRight + +(36) CometProject +Input [5]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38, cs_sold_date_sk#39, d_date_sk#41] +Arguments: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38], [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] + +(37) ColumnarToRow [codegen id : 5] +Input [3]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] + +(38) HashAggregate [codegen id : 5] +Input [3]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] +Keys [1]: [cs_call_center_sk#36] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#37)), partial_sum(UnscaledValue(cs_net_profit#38))] +Aggregate Attributes [2]: [sum#42, sum#43] +Results [3]: [cs_call_center_sk#36, sum#44, sum#45] + +(39) Exchange +Input [3]: [cs_call_center_sk#36, sum#44, sum#45] +Arguments: hashpartitioning(cs_call_center_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(40) HashAggregate [codegen id : 6] +Input [3]: [cs_call_center_sk#36, sum#44, sum#45] +Keys [1]: [cs_call_center_sk#36] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#37)), sum(UnscaledValue(cs_net_profit#38))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#37))#46, sum(UnscaledValue(cs_net_profit#38))#47] +Results [3]: [cs_call_center_sk#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#37))#46,17,2) AS sales#48, MakeDecimal(sum(UnscaledValue(cs_net_profit#38))#47,17,2) AS profit#49] + +(41) BroadcastExchange +Input [3]: [cs_call_center_sk#36, sales#48, profit#49] +Arguments: IdentityBroadcastMode, [plan_id=5] + +(42) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(cr_returned_date_sk#52), dynamicpruningexpression(cr_returned_date_sk#52 IN dynamicpruning#53)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(43) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#54] -(42) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#53] +(44) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52] +Right output [1]: [d_date_sk#54] +Arguments: [cr_returned_date_sk#52], [d_date_sk#54], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#51] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(45) CometProject +Input [4]: [cr_return_amount#50, cr_net_loss#51, cr_returned_date_sk#52, d_date_sk#54] +Arguments: [cr_return_amount#50, cr_net_loss#51], [cr_return_amount#50, cr_net_loss#51] -(44) Project [codegen id : 13] -Output [2]: [cr_return_amount#49, cr_net_loss#50] -Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] +(46) ColumnarToRow [codegen id : 7] +Input [2]: [cr_return_amount#50, cr_net_loss#51] -(45) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#49, cr_net_loss#50] +(47) HashAggregate [codegen id : 7] +Input [2]: [cr_return_amount#50, cr_net_loss#51] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum#54, sum#55] -Results [2]: [sum#56, sum#57] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#50)), partial_sum(UnscaledValue(cr_net_loss#51))] +Aggregate Attributes [2]: [sum#55, sum#56] +Results [2]: [sum#57, sum#58] -(46) Exchange -Input [2]: [sum#56, sum#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(48) Exchange +Input [2]: [sum#57, sum#58] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(47) HashAggregate -Input [2]: [sum#56, sum#57] +(49) HashAggregate +Input [2]: [sum#57, sum#58] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] +Functions [2]: [sum(UnscaledValue(cr_return_amount#50)), sum(UnscaledValue(cr_net_loss#51))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#50))#59, sum(UnscaledValue(cr_net_loss#51))#60] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#50))#59,17,2) AS returns#61, MakeDecimal(sum(UnscaledValue(cr_net_loss#51))#60,17,2) AS profit_loss#62] -(48) BroadcastNestedLoopJoin [codegen id : 14] +(50) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(49) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#62, cs_call_center_sk#35 AS id#63, sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#64] -Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] +(51) Project [codegen id : 8] +Output [5]: [catalog channel AS channel#63, cs_call_center_sk#36 AS id#64, sales#48, returns#61, (profit#49 - profit_loss#62) AS profit#65] +Input [5]: [cs_call_center_sk#36, sales#48, profit#49, returns#61, profit_loss#62] -(50) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(52) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, ws_sold_date_sk#69] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(51) CometFilter -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_web_page_sk#65) - -(52) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(53) CometFilter +Input [4]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, ws_sold_date_sk#69] +Condition : isnotnull(ws_web_page_sk#66) -(53) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#70] +(54) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#71] -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#68] -Right keys [1]: [d_date_sk#70] -Join type: Inner -Join condition: None +(55) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, ws_sold_date_sk#69] +Right output [1]: [d_date_sk#71] +Arguments: [ws_sold_date_sk#69], [d_date_sk#71], Inner, BuildRight -(55) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] -Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] +(56) CometProject +Input [5]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, ws_sold_date_sk#69, d_date_sk#71] +Arguments: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68], [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68] -(56) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#71] +(57) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(57) CometFilter -Input [1]: [wp_web_page_sk#71] -Condition : isnotnull(wp_web_page_sk#71) - -(58) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#71] - -(59) BroadcastExchange -Input [1]: [wp_web_page_sk#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(60) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#65] -Right keys [1]: [wp_web_page_sk#71] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] - -(62) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum#72, sum#73] -Results [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(63) Exchange -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(64) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] -Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] - -(65) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(58) CometFilter +Input [1]: [wp_web_page_sk#72] +Condition : isnotnull(wp_web_page_sk#72) + +(59) CometBroadcastExchange +Input [1]: [wp_web_page_sk#72] +Arguments: [wp_web_page_sk#72] + +(60) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68] +Right output [1]: [wp_web_page_sk#72] +Arguments: [ws_web_page_sk#66], [wp_web_page_sk#72], Inner, BuildRight + +(61) CometProject +Input [4]: [ws_web_page_sk#66, ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] +Arguments: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72], [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] + +(62) ColumnarToRow [codegen id : 9] +Input [3]: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] + +(63) HashAggregate [codegen id : 9] +Input [3]: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] +Keys [1]: [wp_web_page_sk#72] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#67)), partial_sum(UnscaledValue(ws_net_profit#68))] +Aggregate Attributes [2]: [sum#73, sum#74] +Results [3]: [wp_web_page_sk#72, sum#75, sum#76] + +(64) Exchange +Input [3]: [wp_web_page_sk#72, sum#75, sum#76] +Arguments: hashpartitioning(wp_web_page_sk#72, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(65) HashAggregate [codegen id : 12] +Input [3]: [wp_web_page_sk#72, sum#75, sum#76] +Keys [1]: [wp_web_page_sk#72] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#67)), sum(UnscaledValue(ws_net_profit#68))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#67))#77, sum(UnscaledValue(ws_net_profit#68))#78] +Results [3]: [wp_web_page_sk#72, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#67))#77,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(ws_net_profit#68))#78,17,2) AS profit#80] + +(66) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wr_returned_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(wr_returned_date_sk#84), dynamicpruningexpression(wr_returned_date_sk#84 IN dynamicpruning#85)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(66) CometFilter -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] -Condition : isnotnull(wr_web_page_sk#80) +(67) CometFilter +Input [4]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wr_returned_date_sk#84] +Condition : isnotnull(wr_web_page_sk#81) -(67) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(68) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#86] -(68) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#85] +(69) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wr_returned_date_sk#84] +Right output [1]: [d_date_sk#86] +Arguments: [wr_returned_date_sk#84], [d_date_sk#86], Inner, BuildRight -(69) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#83] -Right keys [1]: [d_date_sk#85] -Join type: Inner -Join condition: None - -(70) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] -Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] +(70) CometProject +Input [5]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wr_returned_date_sk#84, d_date_sk#86] +Arguments: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83], [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83] (71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#86] - -(72) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#80] -Right keys [1]: [wp_web_page_sk#86] -Join type: Inner -Join condition: None - -(73) Project [codegen id : 20] -Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] - -(74) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum#87, sum#88] -Results [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(75) Exchange -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(76) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] -Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] - -(77) BroadcastExchange -Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(78) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#71] -Right keys [1]: [wp_web_page_sk#86] +Output [1]: [wp_web_page_sk#87] + +(72) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83] +Right output [1]: [wp_web_page_sk#87] +Arguments: [wr_web_page_sk#81], [wp_web_page_sk#87], Inner, BuildRight + +(73) CometProject +Input [4]: [wr_web_page_sk#81, wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] +Arguments: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87], [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] + +(74) ColumnarToRow [codegen id : 10] +Input [3]: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] + +(75) HashAggregate [codegen id : 10] +Input [3]: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] +Keys [1]: [wp_web_page_sk#87] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#82)), partial_sum(UnscaledValue(wr_net_loss#83))] +Aggregate Attributes [2]: [sum#88, sum#89] +Results [3]: [wp_web_page_sk#87, sum#90, sum#91] + +(76) Exchange +Input [3]: [wp_web_page_sk#87, sum#90, sum#91] +Arguments: hashpartitioning(wp_web_page_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(77) HashAggregate [codegen id : 11] +Input [3]: [wp_web_page_sk#87, sum#90, sum#91] +Keys [1]: [wp_web_page_sk#87] +Functions [2]: [sum(UnscaledValue(wr_return_amt#82)), sum(UnscaledValue(wr_net_loss#83))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#82))#92, sum(UnscaledValue(wr_net_loss#83))#93] +Results [3]: [wp_web_page_sk#87, MakeDecimal(sum(UnscaledValue(wr_return_amt#82))#92,17,2) AS returns#94, MakeDecimal(sum(UnscaledValue(wr_net_loss#83))#93,17,2) AS profit_loss#95] + +(78) BroadcastExchange +Input [3]: [wp_web_page_sk#87, returns#94, profit_loss#95] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +(79) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [wp_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#87] Join type: LeftOuter Join condition: None -(79) Project [codegen id : 22] -Output [5]: [web channel AS channel#95, wp_web_page_sk#71 AS id#96, sales#78, coalesce(returns#93, 0.00) AS returns#97, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#98] -Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] - -(80) Union - -(81) HashAggregate [codegen id : 23] -Input [5]: [channel#31, id#32, sales#14, returns#33, profit#34] -Keys [2]: [channel#31, id#32] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#33), partial_sum(profit#34)] -Aggregate Attributes [6]: [sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(82) Exchange -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(83) HashAggregate [codegen id : 24] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [5]: [channel#31, id#32, cast(sum(sales#14)#111 as decimal(37,2)) AS sales#114, cast(sum(returns#33)#112 as decimal(37,2)) AS returns#115, cast(sum(profit#34)#113 as decimal(38,2)) AS profit#116] - -(84) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] - -(85) HashAggregate [codegen id : 48] -Input [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Keys [2]: [channel#117, id#118] -Functions [3]: [sum(sales#125), sum(returns#126), sum(profit#127)] -Aggregate Attributes [3]: [sum(sales#125)#111, sum(returns#126)#112, sum(profit#127)#113] -Results [4]: [channel#117, sum(sales#125)#111 AS sales#128, sum(returns#126)#112 AS returns#129, sum(profit#127)#113 AS profit#130] - -(86) HashAggregate [codegen id : 48] -Input [4]: [channel#117, sales#128, returns#129, profit#130] -Keys [1]: [channel#117] -Functions [3]: [partial_sum(sales#128), partial_sum(returns#129), partial_sum(profit#130)] -Aggregate Attributes [6]: [sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] -Results [7]: [channel#117, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] - -(87) Exchange -Input [7]: [channel#117, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Arguments: hashpartitioning(channel#117, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(88) HashAggregate [codegen id : 49] -Input [7]: [channel#117, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Keys [1]: [channel#117] -Functions [3]: [sum(sales#128), sum(returns#129), sum(profit#130)] -Aggregate Attributes [3]: [sum(sales#128)#143, sum(returns#129)#144, sum(profit#130)#145] -Results [5]: [channel#117, null AS id#146, sum(sales#128)#143 AS sales#147, sum(returns#129)#144 AS returns#148, sum(profit#130)#145 AS profit#149] - -(89) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#150, id#151, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] - -(90) HashAggregate [codegen id : 73] -Input [8]: [channel#150, id#151, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] -Keys [2]: [channel#150, id#151] -Functions [3]: [sum(sales#158), sum(returns#159), sum(profit#160)] -Aggregate Attributes [3]: [sum(sales#158)#111, sum(returns#159)#112, sum(profit#160)#113] -Results [3]: [sum(sales#158)#111 AS sales#161, sum(returns#159)#112 AS returns#162, sum(profit#160)#113 AS profit#163] - -(91) HashAggregate [codegen id : 73] -Input [3]: [sales#161, returns#162, profit#163] +(80) Project [codegen id : 12] +Output [5]: [web channel AS channel#96, wp_web_page_sk#72 AS id#97, sales#79, coalesce(returns#94, 0.00) AS returns#98, (profit#80 - coalesce(profit_loss#95, 0.00)) AS profit#99] +Input [6]: [wp_web_page_sk#72, sales#79, profit#80, wp_web_page_sk#87, returns#94, profit_loss#95] + +(81) Union + +(82) HashAggregate [codegen id : 13] +Input [5]: [channel#32, id#33, sales#15, returns#34, profit#35] +Keys [2]: [channel#32, id#33] +Functions [3]: [partial_sum(sales#15), partial_sum(returns#34), partial_sum(profit#35)] +Aggregate Attributes [6]: [sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Results [8]: [channel#32, id#33, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] + +(83) Exchange +Input [8]: [channel#32, id#33, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Arguments: hashpartitioning(channel#32, id#33, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(84) HashAggregate [codegen id : 14] +Input [8]: [channel#32, id#33, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Keys [2]: [channel#32, id#33] +Functions [3]: [sum(sales#15), sum(returns#34), sum(profit#35)] +Aggregate Attributes [3]: [sum(sales#15)#112, sum(returns#34)#113, sum(profit#35)#114] +Results [5]: [channel#32, id#33, cast(sum(sales#15)#112 as decimal(37,2)) AS sales#115, cast(sum(returns#34)#113 as decimal(37,2)) AS returns#116, cast(sum(profit#35)#114 as decimal(38,2)) AS profit#117] + +(85) ReusedExchange [Reuses operator id: 83] +Output [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(86) HashAggregate [codegen id : 28] +Input [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Keys [2]: [channel#118, id#119] +Functions [3]: [sum(sales#126), sum(returns#127), sum(profit#128)] +Aggregate Attributes [3]: [sum(sales#126)#112, sum(returns#127)#113, sum(profit#128)#114] +Results [4]: [channel#118, sum(sales#126)#112 AS sales#129, sum(returns#127)#113 AS returns#130, sum(profit#128)#114 AS profit#131] + +(87) HashAggregate [codegen id : 28] +Input [4]: [channel#118, sales#129, returns#130, profit#131] +Keys [1]: [channel#118] +Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] +Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] +Results [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(88) Exchange +Input [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Arguments: hashpartitioning(channel#118, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(89) HashAggregate [codegen id : 29] +Input [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Keys [1]: [channel#118] +Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] +Aggregate Attributes [3]: [sum(sales#129)#144, sum(returns#130)#145, sum(profit#131)#146] +Results [5]: [channel#118, null AS id#147, sum(sales#129)#144 AS sales#148, sum(returns#130)#145 AS returns#149, sum(profit#131)#146 AS profit#150] + +(90) ReusedExchange [Reuses operator id: 83] +Output [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] + +(91) HashAggregate [codegen id : 43] +Input [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Keys [2]: [channel#151, id#152] +Functions [3]: [sum(sales#159), sum(returns#160), sum(profit#161)] +Aggregate Attributes [3]: [sum(sales#159)#112, sum(returns#160)#113, sum(profit#161)#114] +Results [3]: [sum(sales#159)#112 AS sales#162, sum(returns#160)#113 AS returns#163, sum(profit#161)#114 AS profit#164] + +(92) HashAggregate [codegen id : 43] +Input [3]: [sales#162, returns#163, profit#164] Keys: [] -Functions [3]: [partial_sum(sales#161), partial_sum(returns#162), partial_sum(profit#163)] -Aggregate Attributes [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] -Results [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] +Aggregate Attributes [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] +Results [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -(92) Exchange -Input [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] +(93) Exchange +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(93) HashAggregate [codegen id : 74] -Input [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +(94) HashAggregate [codegen id : 44] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys: [] -Functions [3]: [sum(sales#161), sum(returns#162), sum(profit#163)] -Aggregate Attributes [3]: [sum(sales#161)#176, sum(returns#162)#177, sum(profit#163)#178] -Results [5]: [null AS channel#179, null AS id#180, sum(sales#161)#176 AS sales#181, sum(returns#162)#177 AS returns#182, sum(profit#163)#178 AS profit#183] +Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] +Aggregate Attributes [3]: [sum(sales#162)#177, sum(returns#163)#178, sum(profit#164)#179] +Results [5]: [null AS channel#180, null AS id#181, sum(sales#162)#177 AS sales#182, sum(returns#163)#178 AS returns#183, sum(profit#164)#179 AS profit#184] -(94) Union +(95) Union -(95) HashAggregate [codegen id : 75] -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(96) HashAggregate [codegen id : 45] +Input [5]: [channel#32, id#33, sales#115, returns#116, profit#117] +Keys [5]: [channel#32, id#33, sales#115, returns#116, profit#117] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Results [5]: [channel#32, id#33, sales#115, returns#116, profit#117] -(96) Exchange -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: hashpartitioning(channel#31, id#32, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(97) Exchange +Input [5]: [channel#32, id#33, sales#115, returns#116, profit#117] +Arguments: hashpartitioning(channel#32, id#33, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(97) HashAggregate [codegen id : 76] -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(98) HashAggregate [codegen id : 46] +Input [5]: [channel#32, id#33, sales#115, returns#116, profit#117] +Keys [5]: [channel#32, id#33, sales#115, returns#116, profit#117] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Results [5]: [channel#32, id#33, sales#115, returns#116, profit#117] -(98) TakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: 100, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#114, returns#115, profit#116] +(99) TakeOrderedAndProject +Input [5]: [channel#32, id#33, sales#115, returns#116, profit#117] +Arguments: 100, [channel#32 ASC NULLS FIRST, id#33 ASC NULLS FIRST], [channel#32, id#33, sales#115, returns#116, profit#117] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (103) -+- * ColumnarToRow (102) - +- CometProject (101) - +- CometFilter (100) - +- CometScan parquet spark_catalog.default.date_dim (99) +BroadcastExchange (104) ++- * ColumnarToRow (103) + +- CometProject (102) + +- CometFilter (101) + +- CometScan parquet spark_catalog.default.date_dim (100) -(99) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#184] +(100) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_date#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(100) CometFilter -Input [2]: [d_date_sk#6, d_date#184] -Condition : (((isnotnull(d_date#184) AND (d_date#184 >= 1998-08-04)) AND (d_date#184 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +(101) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(101) CometProject -Input [2]: [d_date_sk#6, d_date#184] +(102) CometProject +Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(102) ColumnarToRow [codegen id : 1] +(103) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(103) BroadcastExchange +(104) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#20 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#39 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 40 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#52 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 65 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#84 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt index 670a7e6c3e..141e72805f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt @@ -1,34 +1,34 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (76) + WholeStageCodegen (46) HashAggregate [channel,id,sales,returns,profit] InputAdapter Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) + WholeStageCodegen (45) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (24) + WholeStageCodegen (14) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #2 - WholeStageCodegen (23) + WholeStageCodegen (13) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (8) + WholeStageCodegen (4) Project [s_store_sk,sales,returns,profit,profit_loss] BroadcastHashJoin [s_store_sk,s_store_sk] HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [s_store_sk] #3 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -39,129 +39,116 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (3) HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [s_store_sk] #7 - WholeStageCodegen (6) + Exchange [s_store_sk] #8 + WholeStageCodegen (2) HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometProject [sr_store_sk,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) + ReusedExchange [d_date_sk] #5 + ReusedExchange [s_store_sk] #6 + WholeStageCodegen (8) Project [cs_call_center_sk,sales,returns,profit,profit_loss] BroadcastNestedLoopJoin InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #9 + WholeStageCodegen (6) HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [cs_call_center_sk] #9 - WholeStageCodegen (10) + Exchange [cs_call_center_sk] #10 + WholeStageCodegen (5) HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #5 HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange #10 - WholeStageCodegen (13) + Exchange #11 + WholeStageCodegen (7) HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (12) Project [wp_web_page_sk,sales,returns,profit,profit_loss] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #11 - WholeStageCodegen (17) + Exchange [wp_web_page_sk] #12 + WholeStageCodegen (9) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #13 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) + BroadcastExchange #14 + WholeStageCodegen (11) HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #14 - WholeStageCodegen (20) + Exchange [wp_web_page_sk] #15 + WholeStageCodegen (10) HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] CometFilter [wr_web_page_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) + ReusedExchange [d_date_sk] #5 + ReusedExchange [wp_web_page_sk] #13 + WholeStageCodegen (29) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [channel] #15 - WholeStageCodegen (48) + Exchange [channel] #16 + WholeStageCodegen (28) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) + WholeStageCodegen (44) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange #16 - WholeStageCodegen (73) + Exchange #17 + WholeStageCodegen (43) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt index 58e7f58253..efdca9dca9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt @@ -1,38 +1,40 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * Project (33) - +- Window (32) - +- * Sort (31) - +- Exchange (30) - +- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- Union (26) - :- * HashAggregate (15) - : +- Exchange (14) - : +- * HashAggregate (13) - : +- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - :- * HashAggregate (20) - : +- Exchange (19) - : +- * HashAggregate (18) - : +- * HashAggregate (17) - : +- ReusedExchange (16) - +- * HashAggregate (25) - +- Exchange (24) - +- * HashAggregate (23) - +- * HashAggregate (22) - +- ReusedExchange (21) +TakeOrderedAndProject (36) ++- * Project (35) + +- Window (34) + +- * Sort (33) + +- Exchange (32) + +- * HashAggregate (31) + +- Exchange (30) + +- * HashAggregate (29) + +- Union (28) + :- * HashAggregate (17) + : +- Exchange (16) + : +- * HashAggregate (15) + : +- * ColumnarToRow (14) + : +- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.item (9) + :- * HashAggregate (22) + : +- Exchange (21) + : +- * HashAggregate (20) + : +- * HashAggregate (19) + : +- ReusedExchange (18) + +- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * HashAggregate (24) + +- ReusedExchange (23) (1) Scan parquet spark_catalog.default.web_sales @@ -47,194 +49,205 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 39] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] +Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] -(7) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] +(9) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#7, i_class#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) +(10) CometFilter +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Condition : isnotnull(i_item_sk#7) -(9) ColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#7, i_category#8] +(11) CometBroadcastExchange +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Arguments: [i_item_sk#7, i_class#8, i_category#9] -(10) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#1, ws_net_paid#2] +Right output [3]: [i_item_sk#7, i_class#8, i_category#9] +Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(13) CometProject +Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#8, i_category#9] +Arguments: [ws_net_paid#2, i_class#8, i_category#9], [ws_net_paid#2, i_class#8, i_category#9] -(12) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_class#7, i_category#8] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#7, i_category#8] +(14) ColumnarToRow [codegen id : 1] +Input [3]: [ws_net_paid#2, i_class#8, i_category#9] -(13) HashAggregate [codegen id : 3] -Input [3]: [ws_net_paid#2, i_class#7, i_category#8] -Keys [2]: [i_category#8, i_class#7] +(15) HashAggregate [codegen id : 1] +Input [3]: [ws_net_paid#2, i_class#8, i_category#9] +Keys [2]: [i_category#9, i_class#8] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#9] -Results [3]: [i_category#8, i_class#7, sum#10] +Aggregate Attributes [1]: [sum#10] +Results [3]: [i_category#9, i_class#8, sum#11] -(14) Exchange -Input [3]: [i_category#8, i_class#7, sum#10] -Arguments: hashpartitioning(i_category#8, i_class#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) Exchange +Input [3]: [i_category#9, i_class#8, sum#11] +Arguments: hashpartitioning(i_category#9, i_class#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 4] -Input [3]: [i_category#8, i_class#7, sum#10] -Keys [2]: [i_category#8, i_class#7] +(17) HashAggregate [codegen id : 2] +Input [3]: [i_category#9, i_class#8, sum#11] +Keys [2]: [i_category#9, i_class#8] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) as decimal(27,2)) AS total_sum#12, i_category#8, i_class#7, 0 AS g_category#13, 0 AS g_class#14, 0 AS lochierarchy#15] - -(16) ReusedExchange [Reuses operator id: 14] -Output [3]: [i_category#16, i_class#17, sum#18] - -(17) HashAggregate [codegen id : 8] -Input [3]: [i_category#16, i_class#17, sum#18] -Keys [2]: [i_category#16, i_class#17] -Functions [1]: [sum(UnscaledValue(ws_net_paid#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#19))#11] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#19))#11,17,2) AS total_sum#20, i_category#16] - -(18) HashAggregate [codegen id : 8] -Input [2]: [total_sum#20, i_category#16] -Keys [1]: [i_category#16] -Functions [1]: [partial_sum(total_sum#20)] -Aggregate Attributes [2]: [sum#21, isEmpty#22] -Results [3]: [i_category#16, sum#23, isEmpty#24] - -(19) Exchange -Input [3]: [i_category#16, sum#23, isEmpty#24] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(20) HashAggregate [codegen id : 9] -Input [3]: [i_category#16, sum#23, isEmpty#24] -Keys [1]: [i_category#16] -Functions [1]: [sum(total_sum#20)] -Aggregate Attributes [1]: [sum(total_sum#20)#25] -Results [6]: [sum(total_sum#20)#25 AS total_sum#26, i_category#16, null AS i_class#27, 0 AS g_category#28, 1 AS g_class#29, 1 AS lochierarchy#30] - -(21) ReusedExchange [Reuses operator id: 14] -Output [3]: [i_category#31, i_class#32, sum#33] - -(22) HashAggregate [codegen id : 13] -Input [3]: [i_category#31, i_class#32, sum#33] -Keys [2]: [i_category#31, i_class#32] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#11] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#11,17,2) AS total_sum#35] - -(23) HashAggregate [codegen id : 13] -Input [1]: [total_sum#35] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#12] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#12,17,2) as decimal(27,2)) AS total_sum#13, i_category#9, i_class#8, 0 AS g_category#14, 0 AS g_class#15, 0 AS lochierarchy#16] + +(18) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#17, i_class#18, sum#19] + +(19) HashAggregate [codegen id : 4] +Input [3]: [i_category#17, i_class#18, sum#19] +Keys [2]: [i_category#17, i_class#18] +Functions [1]: [sum(UnscaledValue(ws_net_paid#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#20))#12] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#20))#12,17,2) AS total_sum#21, i_category#17] + +(20) HashAggregate [codegen id : 4] +Input [2]: [total_sum#21, i_category#17] +Keys [1]: [i_category#17] +Functions [1]: [partial_sum(total_sum#21)] +Aggregate Attributes [2]: [sum#22, isEmpty#23] +Results [3]: [i_category#17, sum#24, isEmpty#25] + +(21) Exchange +Input [3]: [i_category#17, sum#24, isEmpty#25] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(22) HashAggregate [codegen id : 5] +Input [3]: [i_category#17, sum#24, isEmpty#25] +Keys [1]: [i_category#17] +Functions [1]: [sum(total_sum#21)] +Aggregate Attributes [1]: [sum(total_sum#21)#26] +Results [6]: [sum(total_sum#21)#26 AS total_sum#27, i_category#17, null AS i_class#28, 0 AS g_category#29, 1 AS g_class#30, 1 AS lochierarchy#31] + +(23) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#32, i_class#33, sum#34] + +(24) HashAggregate [codegen id : 7] +Input [3]: [i_category#32, i_class#33, sum#34] +Keys [2]: [i_category#32, i_class#33] +Functions [1]: [sum(UnscaledValue(ws_net_paid#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#35))#12] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#35))#12,17,2) AS total_sum#36] + +(25) HashAggregate [codegen id : 7] +Input [1]: [total_sum#36] Keys: [] -Functions [1]: [partial_sum(total_sum#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [2]: [sum#38, isEmpty#39] +Functions [1]: [partial_sum(total_sum#36)] +Aggregate Attributes [2]: [sum#37, isEmpty#38] +Results [2]: [sum#39, isEmpty#40] -(24) Exchange -Input [2]: [sum#38, isEmpty#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +(26) Exchange +Input [2]: [sum#39, isEmpty#40] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(25) HashAggregate [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] +(27) HashAggregate [codegen id : 8] +Input [2]: [sum#39, isEmpty#40] Keys: [] -Functions [1]: [sum(total_sum#35)] -Aggregate Attributes [1]: [sum(total_sum#35)#40] -Results [6]: [sum(total_sum#35)#40 AS total_sum#41, null AS i_category#42, null AS i_class#43, 1 AS g_category#44, 1 AS g_class#45, 2 AS lochierarchy#46] +Functions [1]: [sum(total_sum#36)] +Aggregate Attributes [1]: [sum(total_sum#36)#41] +Results [6]: [sum(total_sum#36)#41 AS total_sum#42, null AS i_category#43, null AS i_class#44, 1 AS g_category#45, 1 AS g_class#46, 2 AS lochierarchy#47] -(26) Union +(28) Union -(27) HashAggregate [codegen id : 15] -Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +(29) HashAggregate [codegen id : 9] +Input [6]: [total_sum#13, i_category#9, i_class#8, g_category#14, g_class#15, lochierarchy#16] +Keys [6]: [total_sum#13, i_category#9, i_class#8, g_category#14, g_class#15, lochierarchy#16] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +Results [6]: [total_sum#13, i_category#9, i_class#8, g_category#14, g_class#15, lochierarchy#16] -(28) Exchange -Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -Arguments: hashpartitioning(total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(30) Exchange +Input [6]: [total_sum#13, i_category#9, i_class#8, g_category#14, g_class#15, lochierarchy#16] +Arguments: hashpartitioning(total_sum#13, i_category#9, i_class#8, g_category#14, g_class#15, lochierarchy#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) HashAggregate [codegen id : 16] -Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +(31) HashAggregate [codegen id : 10] +Input [6]: [total_sum#13, i_category#9, i_class#8, g_category#14, g_class#15, lochierarchy#16] +Keys [6]: [total_sum#13, i_category#9, i_class#8, g_category#14, g_class#15, lochierarchy#16] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, CASE WHEN (g_class#14 = 0) THEN i_category#8 END AS _w0#47] +Results [5]: [total_sum#13, i_category#9, i_class#8, lochierarchy#16, CASE WHEN (g_class#15 = 0) THEN i_category#9 END AS _w0#48] -(30) Exchange -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#47] -Arguments: hashpartitioning(lochierarchy#15, _w0#47, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(32) Exchange +Input [5]: [total_sum#13, i_category#9, i_class#8, lochierarchy#16, _w0#48] +Arguments: hashpartitioning(lochierarchy#16, _w0#48, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(31) Sort [codegen id : 17] -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#47] -Arguments: [lochierarchy#15 ASC NULLS FIRST, _w0#47 ASC NULLS FIRST, total_sum#12 DESC NULLS LAST], false, 0 +(33) Sort [codegen id : 11] +Input [5]: [total_sum#13, i_category#9, i_class#8, lochierarchy#16, _w0#48] +Arguments: [lochierarchy#16 ASC NULLS FIRST, _w0#48 ASC NULLS FIRST, total_sum#13 DESC NULLS LAST], false, 0 -(32) Window -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#47] -Arguments: [rank(total_sum#12) windowspecdefinition(lochierarchy#15, _w0#47, total_sum#12 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#48], [lochierarchy#15, _w0#47], [total_sum#12 DESC NULLS LAST] +(34) Window +Input [5]: [total_sum#13, i_category#9, i_class#8, lochierarchy#16, _w0#48] +Arguments: [rank(total_sum#13) windowspecdefinition(lochierarchy#16, _w0#48, total_sum#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#49], [lochierarchy#16, _w0#48], [total_sum#13 DESC NULLS LAST] -(33) Project [codegen id : 18] -Output [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#48] -Input [6]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#47, rank_within_parent#48] +(35) Project [codegen id : 12] +Output [5]: [total_sum#13, i_category#9, i_class#8, lochierarchy#16, rank_within_parent#49] +Input [6]: [total_sum#13, i_category#9, i_class#8, lochierarchy#16, _w0#48, rank_within_parent#49] -(34) TakeOrderedAndProject -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#48] -Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#48 ASC NULLS FIRST], [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#48] +(36) TakeOrderedAndProject +Input [5]: [total_sum#13, i_category#9, i_class#8, lochierarchy#16, rank_within_parent#49] +Arguments: 100, [lochierarchy#16 DESC NULLS LAST, CASE WHEN (lochierarchy#16 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#49 ASC NULLS FIRST], [total_sum#13, i_category#9, i_class#8, lochierarchy#16, rank_within_parent#49] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (39) -+- * ColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan parquet spark_catalog.default.date_dim (35) +BroadcastExchange (41) ++- * ColumnarToRow (40) + +- CometProject (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.date_dim (37) -(35) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#49] +(37) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(36) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#49] -Condition : (((isnotnull(d_month_seq#49) AND (d_month_seq#49 >= 1212)) AND (d_month_seq#49 <= 1223)) AND isnotnull(d_date_sk#5)) +(38) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(37) CometProject -Input [2]: [d_date_sk#5, d_month_seq#49] +(39) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(38) ColumnarToRow [codegen id : 1] +(40) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(39) BroadcastExchange +(41) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt index 5c5e088857..7edc6b80bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt @@ -1,32 +1,32 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (18) + WholeStageCodegen (12) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (17) + WholeStageCodegen (11) Sort [lochierarchy,_w0,total_sum] InputAdapter Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (16) + WholeStageCodegen (10) HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] [_w0] InputAdapter Exchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (15) + WholeStageCodegen (9) HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] InputAdapter Union - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,g_category,g_class,lochierarchy,sum] InputAdapter Exchange [i_category,i_class] #3 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -37,29 +37,27 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (9) + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] InputAdapter - Exchange [i_category] #6 - WholeStageCodegen (8) + Exchange [i_category] #7 + WholeStageCodegen (4) HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (14) + WholeStageCodegen (8) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] InputAdapter - Exchange #7 - WholeStageCodegen (13) + Exchange #8 + WholeStageCodegen (7) HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt index f77b7ec93b..1b71bb0cae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt @@ -1,25 +1,27 @@ == Physical Plan == -* Sort (21) -+- Exchange (20) - +- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +* Sort (23) ++- Exchange (22) + +- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * HashAggregate (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.store_sales @@ -34,122 +36,133 @@ ReadSchema: struct Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [ss_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] +Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(15) HashAggregate [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Aggregate Attributes [1]: [sum#13] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] -(14) Exchange -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +(17) HashAggregate [codegen id : 2] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#14] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] -Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#15] +Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#15,17,2) AS itemrevenue#16, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#15,17,2) AS _w0#17] -(16) Exchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(18) Exchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +(19) Sort [codegen id : 3] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] +(20) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17] +Arguments: [sum(_w0#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9] -(19) Project [codegen id : 6] -Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] -Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] +(21) Project [codegen id : 4] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, ((_w0#17 * 100) / _we0#18) AS revenueratio#19] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, _w0#17, _we0#18] -(20) Exchange -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] +(22) Exchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#19 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) Sort [codegen id : 7] -Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], true, 0 +(23) Sort [codegen id : 5] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#16, revenueratio#19] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#19 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (26) -+- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) -(22) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(23) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(25) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(24) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(26) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(25) ColumnarToRow [codegen id : 1] +(27) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(26) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt index b7489a0aff..2dd86f85b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt @@ -1,27 +1,27 @@ -WholeStageCodegen (7) +WholeStageCodegen (5) Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] InputAdapter Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #2 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -32,12 +32,10 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #5 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index be0e98db20..43d59eb84f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -1,45 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * HashAggregate (40) - +- Exchange (39) - +- * HashAggregate (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * ColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (35) - +- * ColumnarToRow (34) - +- CometFilter (33) - +- CometScan parquet spark_catalog.default.customer_demographics (32) +TakeOrderedAndProject (39) ++- * HashAggregate (38) + +- Exchange (37) + +- * ColumnarToRow (36) + +- CometHashAggregate (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (29) + : +- CometBroadcastHashJoin (28) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometBroadcastHashJoin (11) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : +- CometBroadcastExchange (10) + : : : +- CometProject (9) + : : : +- CometBroadcastHashJoin (8) + : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : +- CometBroadcastExchange (21) + : : +- CometUnion (20) + : : :- CometProject (15) + : : : +- CometBroadcastHashJoin (14) + : : : :- CometScan parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (13) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (17) + : +- CometBroadcastExchange (27) + : +- CometProject (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer_address (24) + +- CometBroadcastExchange (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_demographics (30) (1) Scan parquet spark_catalog.default.customer @@ -53,220 +51,214 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] +ReadSchema: struct + +(5) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) -(6) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#7] +(6) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Arguments: [d_date_sk#7], [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight + +(9) CometProject Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#4] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ss_customer_sk#4] +Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(12) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#11] +(13) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customer_sk#12] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] +(15) CometProject +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +Arguments: [customer_sk#14], [ws_bill_customer_sk#10 AS customer_sk#14] (16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(17) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#18] -(18) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#16] +(18) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Right output [1]: [d_date_sk#18] +Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(19) CometProject +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] +Arguments: [customer_sk#19], [cs_ship_customer_sk#15 AS customer_sk#19] -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customer_sk#17] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] +(20) CometUnion +Child 0 Input [1]: [customer_sk#14] +Child 1 Input [1]: [customer_sk#19] -(21) Union - -(22) BroadcastExchange -Input [1]: [customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(21) CometBroadcastExchange +Input [1]: [customer_sk#14] +Arguments: [customer_sk#14] -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#12] -Join type: LeftSemi -Join condition: None +(22) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [customer_sk#14] +Arguments: [c_customer_sk#1], [customer_sk#14], LeftSemi, BuildRight -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +(23) CometProject Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(25) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_county#19] +(24) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] ReadSchema: struct -(26) CometFilter -Input [2]: [ca_address_sk#18, ca_county#19] -Condition : (ca_county#19 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#18)) +(25) CometFilter +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) -(27) CometProject -Input [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] +(26) CometProject +Input [2]: [ca_address_sk#20, ca_county#21] +Arguments: [ca_address_sk#20], [ca_address_sk#20] -(28) ColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#18] +(27) CometBroadcastExchange +Input [1]: [ca_address_sk#20] +Arguments: [ca_address_sk#20] -(29) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(28) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ca_address_sk#20] +Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None +(29) CometProject +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] -(31) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] - -(32) Scan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(30) Scan parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(33) CometFilter -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#20) - -(34) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(31) CometFilter +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#22) -(35) BroadcastExchange -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(32) CometBroadcastExchange +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(36) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(33) CometBroadcastHashJoin +Left output [1]: [c_current_cdemo_sk#2] +Right output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight -(37) Project [codegen id : 9] -Output [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(34) CometProject +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(38) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(35) CometHashAggregate +Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -(39) Exchange -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(36) ColumnarToRow [codegen id : 1] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#31] -(40) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] -Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +(37) Exchange +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#31] +Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(38) HashAggregate [codegen id : 2] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#31] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] +Aggregate Attributes [1]: [count(1)#32] +Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#32 AS cnt1#33, cd_purchase_estimate#26, count(1)#32 AS cnt2#34, cd_credit_rating#27, count(1)#32 AS cnt3#35, cd_dep_count#28, count(1)#32 AS cnt4#36, cd_dep_employed_count#29, count(1)#32 AS cnt5#37, cd_dep_college_count#30, count(1)#32 AS cnt6#38] -(41) TakeOrderedAndProject -Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] -Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] +(39) TakeOrderedAndProject +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#33, cd_purchase_estimate#26, cnt2#34, cd_credit_rating#27, cnt3#35, cd_dep_count#28, cnt4#36, cd_dep_employed_count#29, cnt5#37, cd_dep_college_count#30, cnt6#38] +Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#33, cd_purchase_estimate#26, cnt2#34, cd_credit_rating#27, cnt3#35, cd_dep_count#28, cnt4#36, cd_dep_employed_count#29, cnt5#37, cd_dep_college_count#30, cnt6#38] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (46) -+- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.date_dim (42) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) -(42) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#38, d_moy#39] +(40) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_moy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter -Input [3]: [d_date_sk#7, d_year#38, d_moy#39] -Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 4)) AND (d_moy#39 <= 7)) AND isnotnull(d_date_sk#7)) +(41) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] +Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) -(44) CometProject -Input [3]: [d_date_sk#7, d_year#38, d_moy#39] +(42) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(45) ColumnarToRow [codegen id : 1] +(43) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(46) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt index 3eb2210a6e..520edc88df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt @@ -1,72 +1,53 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) + WholeStageCodegen (2) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] InputAdapter Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [c_current_cdemo_sk] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,customer_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_county,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometUnion + CometProject [ws_bill_customer_sk] [customer_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometProject [cs_ship_customer_sk] [customer_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #6 + CometProject [ca_address_sk] + CometFilter [ca_county,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange #7 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index daa1f52436..f2b239def2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -2,74 +2,74 @@ TakeOrderedAndProject (71) +- * Project (70) +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * ColumnarToRow (36) - : : : +- CometFilter (35) - : : : +- CometScan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (17) + : : : +- * HashAggregate (16) + : : : +- Exchange (15) + : : : +- * ColumnarToRow (14) + : : : +- CometHashAggregate (13) + : : : +- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- BroadcastExchange (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * ColumnarToRow (31) + : : +- CometHashAggregate (30) + : : +- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- CometBroadcastExchange (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- BroadcastExchange (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * ColumnarToRow (47) + : +- CometHashAggregate (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometProject (42) + : : +- CometBroadcastHashJoin (41) + : : :- CometFilter (37) + : : : +- CometScan parquet spark_catalog.default.customer (36) + : : +- CometBroadcastExchange (40) + : : +- CometFilter (39) + : : +- CometScan parquet spark_catalog.default.web_sales (38) + : +- ReusedExchange (43) +- BroadcastExchange (68) +- * HashAggregate (67) +- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * ColumnarToRow (55) - : : +- CometFilter (54) - : : +- CometScan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) + +- * ColumnarToRow (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometBroadcastHashJoin (62) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (61) (1) Scan parquet spark_catalog.default.customer @@ -83,10 +83,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] Condition : isnotnull(ss_customer_sk#9) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(7) BroadcastExchange -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Right output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: [c_customer_sk#1], [ss_customer_sk#9], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +(7) CometProject Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -(10) ReusedExchange [Reuses operator id: 75] +(8) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_year#15] +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 -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) + +(10) CometBroadcastExchange +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14, d_year#15] -(12) Project [codegen id : 3] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] +(11) CometBroadcastHashJoin +Left output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Right output [2]: [d_date_sk#14, d_year#15] +Arguments: [ss_sold_date_sk#12], [d_date_sk#14], Inner, BuildRight + +(12) CometProject Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#14, d_year#15] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] -(13) HashAggregate [codegen id : 3] +(13) CometHashAggregate Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] -Aggregate Attributes [1]: [sum#16] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -(14) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(14) ColumnarToRow [codegen id : 1] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#16] + +(15) Exchange +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#16] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 16] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] +(16) HashAggregate [codegen id : 8] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#16] Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] -Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#17] +Results [2]: [c_customer_id#2 AS customer_id#18, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#17,18,2) AS year_total#19] -(16) Filter [codegen id : 16] -Input [2]: [customer_id#19, year_total#20] -Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) +(17) Filter [codegen id : 8] +Input [2]: [customer_id#18, year_total#19] +Condition : (isnotnull(year_total#19) AND (year_total#19 > 0.00)) -(17) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +(18) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter -Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) - -(19) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +(19) CometFilter +Input [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_customer_id#21)) (20) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#33)] +PartitionFilters: [isnotnull(ss_sold_date_sk#31), dynamicpruningexpression(ss_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (21) CometFilter -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -Condition : isnotnull(ss_customer_sk#29) +Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Condition : isnotnull(ss_customer_sk#28) -(22) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +(22) CometBroadcastExchange +Input [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Arguments: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -(23) BroadcastExchange -Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Right output [4]: [ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Arguments: [c_customer_sk#20], [ss_customer_sk#28], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#21] -Right keys [1]: [ss_customer_sk#29] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] -Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] - -(26) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#34, d_year#35] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(24) CometProject +Input [12]: [c_customer_sk#20, c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_customer_sk#28, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31], [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] -Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] - -(29) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] -Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] -Aggregate Attributes [1]: [sum#36] -Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] - -(30) Exchange -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] -Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] -Results [5]: [c_customer_id#22 AS customer_id#38, c_first_name#23 AS customer_first_name#39, c_last_name#24 AS customer_last_name#40, c_email_address#28 AS customer_email_address#41, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#42] - -(32) BroadcastExchange -Input [5]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#33, d_year#34] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct -(33) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#38] +(26) CometFilter +Input [2]: [d_date_sk#33, d_year#34] +Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) + +(27) CometBroadcastExchange +Input [2]: [d_date_sk#33, d_year#34] +Arguments: [d_date_sk#33, d_year#34] + +(28) CometBroadcastHashJoin +Left output [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31] +Right output [2]: [d_date_sk#33, d_year#34] +Arguments: [ss_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight + +(29) CometProject +Input [12]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, ss_sold_date_sk#31, d_date_sk#33, d_year#34] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34], [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] + +(30) CometHashAggregate +Input [10]: [c_customer_id#21, c_first_name#22, c_last_name#23, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, ss_ext_discount_amt#29, ss_ext_list_price#30, d_year#34] +Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] + +(31) ColumnarToRow [codegen id : 2] +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#35] + +(32) Exchange +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#35] +Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(33) HashAggregate [codegen id : 3] +Input [9]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27, sum#35] +Keys [8]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#34, c_preferred_cust_flag#24, c_birth_country#25, c_login#26, c_email_address#27] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))#17] +Results [5]: [c_customer_id#21 AS customer_id#36, c_first_name#22 AS customer_first_name#37, c_last_name#23 AS customer_last_name#38, c_email_address#27 AS customer_email_address#39, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#30 - ss_ext_discount_amt#29)))#17,18,2) AS year_total#40] + +(34) BroadcastExchange +Input [5]: [customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39, year_total#40] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(35) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#36] Join type: Inner Join condition: None -(34) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] +(36) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter -Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] -Condition : (isnotnull(c_customer_sk#43) AND isnotnull(c_customer_id#44)) - -(36) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] +(37) CometFilter +Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_customer_id#42)) -(37) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +(38) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] +PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#53)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#51) +(39) CometFilter +Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Condition : isnotnull(ws_bill_customer_sk#49) -(39) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +(40) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Arguments: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -(40) BroadcastExchange -Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(41) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Right output [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Arguments: [c_customer_sk#41], [ws_bill_customer_sk#49], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#43] -Right keys [1]: [ws_bill_customer_sk#51] -Join type: Inner -Join condition: None +(42) CometProject +Input [12]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Arguments: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52], [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] -(42) Project [codegen id : 10] -Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] -Input [12]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +(43) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#54, d_year#55] -(43) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#56, d_year#57] +(44) CometBroadcastHashJoin +Left output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Right output [2]: [d_date_sk#54, d_year#55] +Arguments: [ws_sold_date_sk#52], [d_date_sk#54], Inner, BuildRight -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#54] -Right keys [1]: [d_date_sk#56] -Join type: Inner -Join condition: None +(45) CometProject +Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52, d_date_sk#54, d_year#55] +Arguments: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55], [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] + +(46) CometHashAggregate +Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] + +(47) ColumnarToRow [codegen id : 4] +Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#56] -(45) Project [codegen id : 10] -Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] -Input [12]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] - -(46) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] -Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] -Aggregate Attributes [1]: [sum#58] -Results [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] - -(47) Exchange -Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] -Arguments: hashpartitioning(c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(48) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] -Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60] -Results [2]: [c_customer_id#44 AS customer_id#61, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60,18,2) AS year_total#62] - -(49) Filter [codegen id : 11] -Input [2]: [customer_id#61, year_total#62] -Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.00)) - -(50) BroadcastExchange -Input [2]: [customer_id#61, year_total#62] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#61] +(48) Exchange +Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#56] +Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(49) HashAggregate [codegen id : 5] +Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#56] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#57] +Results [2]: [c_customer_id#42 AS customer_id#58, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#57,18,2) AS year_total#59] + +(50) Filter [codegen id : 5] +Input [2]: [customer_id#58, year_total#59] +Condition : (isnotnull(year_total#59) AND (year_total#59 > 0.00)) + +(51) BroadcastExchange +Input [2]: [customer_id#58, year_total#59] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(52) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#58] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [8]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62] -Input [9]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, customer_id#61, year_total#62] +(53) Project [codegen id : 8] +Output [8]: [customer_id#18, year_total#19, customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39, year_total#40, year_total#59] +Input [9]: [customer_id#18, year_total#19, customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39, year_total#40, customer_id#58, year_total#59] -(53) Scan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] +(54) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter -Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] -Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_customer_id#64)) - -(55) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] +(55) CometFilter +Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] +Condition : (isnotnull(c_customer_sk#60) AND isnotnull(c_customer_id#61)) (56) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +Output [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ws_sold_date_sk#71), dynamicpruningexpression(ws_sold_date_sk#71 IN dynamicpruning#72)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (57) CometFilter -Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] -Condition : isnotnull(ws_bill_customer_sk#71) +Input [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] +Condition : isnotnull(ws_bill_customer_sk#68) -(58) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +(58) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] +Arguments: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] -(59) BroadcastExchange -Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(59) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] +Right output [4]: [ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] +Arguments: [c_customer_sk#60], [ws_bill_customer_sk#68], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#63] -Right keys [1]: [ws_bill_customer_sk#71] -Join type: Inner -Join condition: None +(60) CometProject +Input [12]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_bill_customer_sk#68, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] +Arguments: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71], [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] -(61) Project [codegen id : 14] -Output [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] -Input [12]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +(61) ReusedExchange [Reuses operator id: 27] +Output [2]: [d_date_sk#73, d_year#74] -(62) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#76, d_year#77] +(62) CometBroadcastHashJoin +Left output [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71] +Right output [2]: [d_date_sk#73, d_year#74] +Arguments: [ws_sold_date_sk#71], [d_date_sk#73], Inner, BuildRight -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#74] -Right keys [1]: [d_date_sk#76] -Join type: Inner -Join condition: None +(63) CometProject +Input [12]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, ws_sold_date_sk#71, d_date_sk#73, d_year#74] +Arguments: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, d_year#74], [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, d_year#74] -(64) Project [codegen id : 14] -Output [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] -Input [12]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74, d_date_sk#76, d_year#77] +(64) CometHashAggregate +Input [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, d_year#74] +Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))] -(65) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] -Keys [8]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))] -Aggregate Attributes [1]: [sum#78] -Results [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] +(65) ColumnarToRow [codegen id : 6] +Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74, sum#75] (66) Exchange -Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] -Arguments: hashpartitioning(c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74, sum#75] +Arguments: hashpartitioning(c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(67) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] -Keys [8]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))#60] -Results [2]: [c_customer_id#64 AS customer_id#80, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))#60,18,2) AS year_total#81] +(67) HashAggregate [codegen id : 7] +Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74, sum#75] +Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#74] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))#57] +Results [2]: [c_customer_id#61 AS customer_id#76, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))#57,18,2) AS year_total#77] (68) BroadcastExchange -Input [2]: [customer_id#80, year_total#81] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +Input [2]: [customer_id#76, year_total#77] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(69) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#19] -Right keys [1]: [customer_id#80] +(69) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#18] +Right keys [1]: [customer_id#76] Join type: Inner -Join condition: (CASE WHEN (year_total#62 > 0.00) THEN (year_total#81 / year_total#62) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END) +Join condition: (CASE WHEN (year_total#59 > 0.00) THEN (year_total#77 / year_total#59) ELSE 0E-20 END > CASE WHEN (year_total#19 > 0.00) THEN (year_total#40 / year_total#19) ELSE 0E-20 END) -(70) Project [codegen id : 16] -Output [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] -Input [10]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62, customer_id#80, year_total#81] +(70) Project [codegen id : 8] +Output [4]: [customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39] +Input [10]: [customer_id#18, year_total#19, customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39, year_total#40, year_total#59, customer_id#76, year_total#77] (71) TakeOrderedAndProject -Input [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] -Arguments: 100, [customer_id#38 ASC NULLS FIRST, customer_first_name#39 ASC NULLS FIRST, customer_last_name#40 ASC NULLS FIRST, customer_email_address#41 ASC NULLS FIRST], [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] +Input [4]: [customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39] +Arguments: 100, [customer_id#36 ASC NULLS FIRST, customer_first_name#37 ASC NULLS FIRST, customer_last_name#38 ASC NULLS FIRST, customer_email_address#39 ASC NULLS FIRST], [customer_id#36, customer_first_name#37, customer_last_name#38, customer_email_address#39] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (75) +- * ColumnarToRow (74) +- CometFilter (73) @@ -443,9 +439,9 @@ Input [2]: [d_date_sk#14, d_year#15] (75) BroadcastExchange Input [2]: [d_date_sk#14, d_year#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#31 IN dynamicpruning#32 BroadcastExchange (79) +- * ColumnarToRow (78) +- CometFilter (77) @@ -453,25 +449,25 @@ BroadcastExchange (79) (76) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#34, d_year#35] +Output [2]: [d_date_sk#33, d_year#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct (77) CometFilter -Input [2]: [d_date_sk#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) +Input [2]: [d_date_sk#33, d_year#34] +Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2002)) AND isnotnull(d_date_sk#33)) (78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_year#35] +Input [2]: [d_date_sk#33, d_year#34] (79) BroadcastExchange -Input [2]: [d_date_sk#34, d_year#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Input [2]: [d_date_sk#33, d_year#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#33 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#71 IN dynamicpruning#32 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt index 0a30aba051..ecc421bd51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - WholeStageCodegen (16) + WholeStageCodegen (8) Project [customer_id,customer_first_name,customer_last_name,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] @@ -9,114 +9,98 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + CometBroadcastExchange #7 CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #13 + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index c39a71879d..04b5d41b59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * ColumnarToRow (15) + +- CometHashAggregate (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.web_sales @@ -33,118 +35,127 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [ws_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) CometHashAggregate Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(15) ColumnarToRow [codegen id : 1] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16] -(16) Exchange +(18) Exchange Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] +(19) Sort [codegen id : 3] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window +(20) Window Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(19) Project [codegen id : 6] +(21) Project [codegen id : 4] Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] -(20) TakeOrderedAndProject +(22) TakeOrderedAndProject Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) -(21) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(23) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 1bc2538b48..545f0ececc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,12 +29,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 1e7dfdbdf7..cded32ec2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -23,43 +23,43 @@ TakeOrderedAndProject (84) : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) : : : :- * HashAggregate (35) : : : : +- Exchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * ColumnarToRow (9) - : : : : : : +- CometFilter (8) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * ColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * ColumnarToRow (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) + : : : : +- * ColumnarToRow (33) + : : : : +- CometHashAggregate (32) + : : : : +- CometProject (31) + : : : : +- CometBroadcastHashJoin (30) + : : : : :- CometProject (28) + : : : : : +- CometBroadcastHashJoin (27) + : : : : : :- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- CometBroadcastExchange (26) + : : : : : +- CometBroadcastHashJoin (25) + : : : : : :- CometFilter (10) + : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : +- CometBroadcastExchange (24) + : : : : : +- CometProject (23) + : : : : : +- CometBroadcastHashJoin (22) + : : : : : :- CometProject (17) + : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : :- CometFilter (12) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : +- CometBroadcastExchange (15) + : : : : : : +- CometFilter (14) + : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : +- CometBroadcastExchange (21) + : : : : : +- CometProject (20) + : : : : : +- CometFilter (19) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) + : : : : +- ReusedExchange (29) : : : +- BroadcastExchange (45) - : : : +- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * ColumnarToRow (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : +- ReusedExchange (39) - : : : +- ReusedExchange (42) + : : : +- * ColumnarToRow (44) + : : : +- CometProject (43) + : : : +- CometBroadcastHashJoin (42) + : : : :- CometProject (40) + : : : : +- CometBroadcastHashJoin (39) + : : : : :- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (38) + : : : +- ReusedExchange (41) : : +- BroadcastExchange (57) : : +- * BroadcastHashJoin LeftSemi BuildRight (56) : : :- * ColumnarToRow (54) @@ -97,7 +97,7 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 25] +(3) ColumnarToRow [codegen id : 11] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (4) Scan parquet spark_catalog.default.item @@ -111,7 +111,7 @@ ReadSchema: struct Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -151,605 +145,599 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(19) BroadcastExchange +(15) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(17) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -(22) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#24] +(18) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(19) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(20) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] + +(21) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] + +(22) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight + +(23) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(24) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(25) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(28) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#25] +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(31) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] (34) Exchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] +Results [3]: [brand_id#27, class_id#28, category_id#29] (36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (37) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(42) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#36] +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] (45) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(48) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(49) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] (50) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [ss_item_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (52) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (53) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) -(54) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] (55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (57) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(58) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(59) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#42] +(60) ReusedExchange [Reuses operator id: 106] +Output [1]: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(62) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(63) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] (64) Exchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(65) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(65) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [6]: [store AS channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#53, count(1)#51 AS number_sales#54] -(66) Filter [codegen id : 52] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(66) Filter [codegen id : 24] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Condition : (isnotnull(sales#53) AND (cast(sales#53 as decimal(32,6)) > cast(Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (67) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Output [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (68) CometFilter -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] -Condition : isnotnull(ss_item_sk#56) +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] +Condition : isnotnull(ss_item_sk#57) -(69) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +(69) ColumnarToRow [codegen id : 22] +Input [4]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60] (70) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(71) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [ss_item_sk#37] +(71) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (72) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#56] -Right keys [1]: [i_item_sk#61] +(73) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_item_sk#57] +Right keys [1]: [i_item_sk#62] Join type: Inner Join condition: None -(74) Project [codegen id : 50] -Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +(74) Project [codegen id : 22] +Output [6]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [ss_item_sk#57, ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(75) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#65] +(75) ReusedExchange [Reuses operator id: 120] +Output [1]: [d_date_sk#66] -(76) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#65] +(76) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ss_sold_date_sk#60] +Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(77) Project [codegen id : 50] -Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] +(77) Project [codegen id : 22] +Output [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [ss_quantity#58, ss_list_price#59, ss_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] -(78) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] -Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +(78) HashAggregate [codegen id : 22] +Input [5]: [ss_quantity#58, ss_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] (79) Exchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(80) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72, count(1)#73] -Results [6]: [store AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] +(80) HashAggregate [codegen id : 23] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#73, count(1)#74] +Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#58 as decimal(10,0)) * ss_list_price#59))#73 AS sales#76, count(1)#74 AS number_sales#77] -(81) Filter [codegen id : 51] -Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(81) Filter [codegen id : 23] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (82) BroadcastExchange -Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=8] -(83) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Right keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] +(83) BroadcastHashJoin [codegen id : 24] +Left keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] +Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Join type: Inner Join condition: None (84) TakeOrderedAndProject -Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] +Input [12]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: 100, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (103) -+- Exchange (102) - +- * HashAggregate (101) - +- Union (100) - :- * Project (89) - : +- * BroadcastHashJoin Inner BuildRight (88) - : :- * ColumnarToRow (86) - : : +- CometScan parquet spark_catalog.default.store_sales (85) - : +- ReusedExchange (87) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * ColumnarToRow (91) - : : +- CometScan parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (92) - +- * Project (99) - +- * BroadcastHashJoin Inner BuildRight (98) - :- * ColumnarToRow (96) - : +- CometScan parquet spark_catalog.default.web_sales (95) - +- ReusedExchange (97) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#55, [id=#56] +* HashAggregate (101) ++- Exchange (100) + +- * ColumnarToRow (99) + +- CometHashAggregate (98) + +- CometUnion (97) + :- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometScan parquet spark_catalog.default.store_sales (85) + : +- ReusedExchange (86) + :- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometScan parquet spark_catalog.default.catalog_sales (89) + : +- ReusedExchange (90) + +- CometProject (96) + +- CometBroadcastHashJoin (95) + :- CometScan parquet spark_catalog.default.web_sales (93) + +- ReusedExchange (94) (85) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] +Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#79), dynamicpruningexpression(ss_sold_date_sk#79 IN dynamicpruning#80)] +PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79] - -(87) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#81] +(86) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#82] -(88) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#79] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None +(87) CometBroadcastHashJoin +Left output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] +Right output [1]: [d_date_sk#82] +Arguments: [ss_sold_date_sk#80], [d_date_sk#82], Inner, BuildRight -(89) Project [codegen id : 2] -Output [2]: [ss_quantity#77 AS quantity#82, ss_list_price#78 AS list_price#83] -Input [4]: [ss_quantity#77, ss_list_price#78, ss_sold_date_sk#79, d_date_sk#81] +(88) CometProject +Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] +Arguments: [quantity#83, list_price#84], [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] -(90) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] +(89) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#86), dynamicpruningexpression(cs_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] ReadSchema: struct -(91) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86] +(90) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#89] -(92) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#88] +(91) CometBroadcastHashJoin +Left output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +Right output [1]: [d_date_sk#89] +Arguments: [cs_sold_date_sk#87], [d_date_sk#89], Inner, BuildRight -(93) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#86] -Right keys [1]: [d_date_sk#88] -Join type: Inner -Join condition: None +(92) CometProject +Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] +Arguments: [quantity#90, list_price#91], [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] -(94) Project [codegen id : 4] -Output [2]: [cs_quantity#84 AS quantity#89, cs_list_price#85 AS list_price#90] -Input [4]: [cs_quantity#84, cs_list_price#85, cs_sold_date_sk#86, d_date_sk#88] - -(95) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +(93) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] ReadSchema: struct -(96) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93] +(94) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#96] -(97) ReusedExchange [Reuses operator id: 117] -Output [1]: [d_date_sk#95] +(95) CometBroadcastHashJoin +Left output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Right output [1]: [d_date_sk#96] +Arguments: [ws_sold_date_sk#94], [d_date_sk#96], Inner, BuildRight -(98) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#93] -Right keys [1]: [d_date_sk#95] -Join type: Inner -Join condition: None +(96) CometProject +Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] +Arguments: [quantity#97, list_price#98], [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] -(99) Project [codegen id : 6] -Output [2]: [ws_quantity#91 AS quantity#96, ws_list_price#92 AS list_price#97] -Input [4]: [ws_quantity#91, ws_list_price#92, ws_sold_date_sk#93, d_date_sk#95] +(97) CometUnion +Child 0 Input [2]: [quantity#83, list_price#84] +Child 1 Input [2]: [quantity#90, list_price#91] +Child 2 Input [2]: [quantity#97, list_price#98] -(100) Union - -(101) HashAggregate [codegen id : 7] -Input [2]: [quantity#82, list_price#83] +(98) CometHashAggregate +Input [2]: [quantity#83, list_price#84] Keys: [] -Functions [1]: [partial_avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] -Aggregate Attributes [2]: [sum#98, count#99] -Results [2]: [sum#100, count#101] +Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] + +(99) ColumnarToRow [codegen id : 1] +Input [2]: [sum#99, count#100] -(102) Exchange -Input [2]: [sum#100, count#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] +(100) Exchange +Input [2]: [sum#99, count#100] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(103) HashAggregate [codegen id : 8] -Input [2]: [sum#100, count#101] +(101) HashAggregate [codegen id : 2] +Input [2]: [sum#99, count#100] Keys: [] -Functions [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))] -Aggregate Attributes [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102] -Results [1]: [avg((cast(quantity#82 as decimal(10,0)) * list_price#83))#102 AS average_sales#103] +Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#101] +Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#101 AS average_sales#102] -Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#79 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#86 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 89 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * ColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan parquet spark_catalog.default.date_dim (104) +BroadcastExchange (106) ++- * ColumnarToRow (105) + +- CometProject (104) + +- CometFilter (103) + +- CometScan parquet spark_catalog.default.date_dim (102) -(104) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#42, d_week_seq#104] +(102) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#43, d_week_seq#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(105) CometFilter -Input [2]: [d_date_sk#42, d_week_seq#104] -Condition : ((isnotnull(d_week_seq#104) AND (d_week_seq#104 = Subquery scalar-subquery#105, [id=#106])) AND isnotnull(d_date_sk#42)) +(103) CometFilter +Input [2]: [d_date_sk#43, d_week_seq#103] +Condition : ((isnotnull(d_week_seq#103) AND (d_week_seq#103 = Subquery scalar-subquery#104, [id=#105])) AND isnotnull(d_date_sk#43)) -(106) CometProject -Input [2]: [d_date_sk#42, d_week_seq#104] -Arguments: [d_date_sk#42], [d_date_sk#42] +(104) CometProject +Input [2]: [d_date_sk#43, d_week_seq#103] +Arguments: [d_date_sk#43], [d_date_sk#43] -(107) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(105) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(108) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(106) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:6 Hosting operator id = 105 Hosting Expression = Subquery scalar-subquery#105, [id=#106] -* ColumnarToRow (112) -+- CometProject (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +Subquery:6 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#104, [id=#105] +* ColumnarToRow (110) ++- CometProject (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(109) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] +(107) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(110) CometFilter -Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] -Condition : (((((isnotnull(d_year#108) AND isnotnull(d_moy#109)) AND isnotnull(d_dom#110)) AND (d_year#108 = 1999)) AND (d_moy#109 = 12)) AND (d_dom#110 = 16)) +(108) CometFilter +Input [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] +Condition : (((((isnotnull(d_year#107) AND isnotnull(d_moy#108)) AND isnotnull(d_dom#109)) AND (d_year#107 = 1999)) AND (d_moy#108 = 12)) AND (d_dom#109 = 16)) -(111) CometProject -Input [4]: [d_week_seq#107, d_year#108, d_moy#109, d_dom#110] -Arguments: [d_week_seq#107], [d_week_seq#107] +(109) CometProject +Input [4]: [d_week_seq#106, d_year#107, d_moy#108, d_dom#109] +Arguments: [d_week_seq#106], [d_week_seq#106] -(112) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#107] +(110) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#106] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (115) ++- * ColumnarToRow (114) + +- CometProject (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) -(113) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#111] +(111) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#110] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(114) CometFilter -Input [2]: [d_date_sk#25, d_year#111] -Condition : (((isnotnull(d_year#111) AND (d_year#111 >= 1998)) AND (d_year#111 <= 2000)) AND isnotnull(d_date_sk#25)) +(112) CometFilter +Input [2]: [d_date_sk#26, d_year#110] +Condition : (((isnotnull(d_year#110) AND (d_year#110 >= 1998)) AND (d_year#110 <= 2000)) AND isnotnull(d_date_sk#26)) -(115) CometProject -Input [2]: [d_date_sk#25, d_year#111] -Arguments: [d_date_sk#25], [d_date_sk#25] +(113) CometProject +Input [2]: [d_date_sk#26, d_year#110] +Arguments: [d_date_sk#26], [d_date_sk#26] -(116) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(114) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(117) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] +(115) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:9 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:10 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 -BroadcastExchange (122) -+- * ColumnarToRow (121) - +- CometProject (120) - +- CometFilter (119) - +- CometScan parquet spark_catalog.default.date_dim (118) +Subquery:11 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#60 IN dynamicpruning#61 +BroadcastExchange (120) ++- * ColumnarToRow (119) + +- CometProject (118) + +- CometFilter (117) + +- CometScan parquet spark_catalog.default.date_dim (116) -(118) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#112] +(116) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#66, d_week_seq#111] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(119) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#112] -Condition : ((isnotnull(d_week_seq#112) AND (d_week_seq#112 = Subquery scalar-subquery#113, [id=#114])) AND isnotnull(d_date_sk#65)) +(117) CometFilter +Input [2]: [d_date_sk#66, d_week_seq#111] +Condition : ((isnotnull(d_week_seq#111) AND (d_week_seq#111 = Subquery scalar-subquery#112, [id=#113])) AND isnotnull(d_date_sk#66)) -(120) CometProject -Input [2]: [d_date_sk#65, d_week_seq#112] -Arguments: [d_date_sk#65], [d_date_sk#65] +(118) CometProject +Input [2]: [d_date_sk#66, d_week_seq#111] +Arguments: [d_date_sk#66], [d_date_sk#66] -(121) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] +(119) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#66] -(122) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +(120) BroadcastExchange +Input [1]: [d_date_sk#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -Subquery:12 Hosting operator id = 119 Hosting Expression = Subquery scalar-subquery#113, [id=#114] -* ColumnarToRow (126) -+- CometProject (125) - +- CometFilter (124) - +- CometScan parquet spark_catalog.default.date_dim (123) +Subquery:12 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#112, [id=#113] +* ColumnarToRow (124) ++- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) -(123) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] +(121) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(124) CometFilter -Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] -Condition : (((((isnotnull(d_year#116) AND isnotnull(d_moy#117)) AND isnotnull(d_dom#118)) AND (d_year#116 = 1998)) AND (d_moy#117 = 12)) AND (d_dom#118 = 16)) +(122) CometFilter +Input [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] +Condition : (((((isnotnull(d_year#115) AND isnotnull(d_moy#116)) AND isnotnull(d_dom#117)) AND (d_year#115 = 1998)) AND (d_moy#116 = 12)) AND (d_dom#117 = 16)) -(125) CometProject -Input [4]: [d_week_seq#115, d_year#116, d_moy#117, d_dom#118] -Arguments: [d_week_seq#115], [d_week_seq#115] +(123) CometProject +Input [4]: [d_week_seq#114, d_year#115, d_moy#116, d_dom#117] +Arguments: [d_week_seq#114], [d_week_seq#114] -(126) ColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#115] +(124) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#114] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 09d8d9dde3..799f74a369 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -1,47 +1,36 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (24) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] + Exchange #13 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 + CometHashAggregate [quantity,list_price] + CometUnion + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk] #10 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (11) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -69,7 +58,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #3 - WholeStageCodegen (11) + WholeStageCodegen (4) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow @@ -78,19 +67,19 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (10) + WholeStageCodegen (3) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 @@ -101,56 +90,43 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 + CometBroadcastExchange #7 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #8 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + CometBroadcastExchange #9 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #10 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #11 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + ReusedExchange [d_date_sk] #10 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) + BroadcastExchange #12 + WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter @@ -161,14 +137,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) + BroadcastExchange #14 + WholeStageCodegen (23) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (22) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -180,7 +156,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 + BroadcastExchange #16 WholeStageCodegen (1) ColumnarToRow InputAdapter @@ -197,6 +173,6 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter - ReusedExchange [d_date_sk] #15 + ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index 88d8caaa15..6119803928 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -30,43 +30,43 @@ TakeOrderedAndProject (125) : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) : : : : :- * HashAggregate (35) : : : : : +- Exchange (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * ColumnarToRow (9) - : : : : : : : +- CometFilter (8) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * ColumnarToRow (12) - : : : : : : : +- CometFilter (11) - : : : : : : : +- CometScan parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * ColumnarToRow (15) - : : : : : : : : +- CometFilter (14) - : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) + : : : : : +- * ColumnarToRow (33) + : : : : : +- CometHashAggregate (32) + : : : : : +- CometProject (31) + : : : : : +- CometBroadcastHashJoin (30) + : : : : : :- CometProject (28) + : : : : : : +- CometBroadcastHashJoin (27) + : : : : : : :- CometFilter (8) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : : +- CometBroadcastExchange (26) + : : : : : : +- CometBroadcastHashJoin (25) + : : : : : : :- CometFilter (10) + : : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : : +- CometBroadcastExchange (24) + : : : : : : +- CometProject (23) + : : : : : : +- CometBroadcastHashJoin (22) + : : : : : : :- CometProject (17) + : : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : : :- CometFilter (12) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : : : : : : : +- CometBroadcastExchange (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : : +- CometBroadcastExchange (21) + : : : : : : +- CometProject (20) + : : : : : : +- CometFilter (19) + : : : : : : +- CometScan parquet spark_catalog.default.date_dim (18) + : : : : : +- ReusedExchange (29) : : : : +- BroadcastExchange (45) - : : : : +- * Project (44) - : : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : : :- * Project (41) - : : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : : :- * ColumnarToRow (38) - : : : : : : +- CometFilter (37) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) - : : : : : +- ReusedExchange (39) - : : : : +- ReusedExchange (42) + : : : : +- * ColumnarToRow (44) + : : : : +- CometProject (43) + : : : : +- CometBroadcastHashJoin (42) + : : : : :- CometProject (40) + : : : : : +- CometBroadcastHashJoin (39) + : : : : : :- CometFilter (37) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : : +- ReusedExchange (38) + : : : : +- ReusedExchange (41) : : : +- BroadcastExchange (57) : : : +- * BroadcastHashJoin LeftSemi BuildRight (56) : : : :- * ColumnarToRow (54) @@ -138,7 +138,7 @@ ReadSchema: struct Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 25] +(3) ColumnarToRow [codegen id : 11] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (4) Scan parquet spark_catalog.default.item @@ -152,7 +152,7 @@ ReadSchema: struct Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) -(9) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(10) Scan parquet spark_catalog.default.item +(9) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(12) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales +(11) Scan parquet spark_catalog.default.catalog_sales Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] @@ -192,773 +186,786 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) -(15) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] Condition : isnotnull(i_item_sk#20) -(18) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(19) BroadcastExchange +(15) CometBroadcastExchange Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#20] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +(17) CometProject Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] + +(18) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(22) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#24] +(19) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(20) CometProject +Input [2]: [d_date_sk#24, d_year#25] +Arguments: [d_date_sk#24], [d_date_sk#24] + +(21) CometBroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: [d_date_sk#24] -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +(22) CometBroadcastHashJoin +Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Right output [1]: [d_date_sk#24] +Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight + +(23) CometProject Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] -(25) BroadcastExchange +(24) CometBroadcastExchange Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] +Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] -Join type: LeftSemi -Join condition: None +(25) CometBroadcastHashJoin +Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight -(27) BroadcastExchange +(26) CometBroadcastExchange Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +(28) CometProject Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -(30) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#25] +(29) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#26] -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Right output [1]: [d_date_sk#26] +Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] +(31) CometProject +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] +Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(32) CometHashAggregate +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] + +(33) ColumnarToRow [codegen id : 1] +Input [3]: [brand_id#27, class_id#28, category_id#29] (34) Exchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(35) HashAggregate [codegen id : 10] -Input [3]: [brand_id#26, class_id#27, category_id#28] -Keys [3]: [brand_id#26, class_id#27, category_id#28] +(35) HashAggregate [codegen id : 3] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Keys [3]: [brand_id#27, class_id#28, category_id#29] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#26, class_id#27, category_id#28] +Results [3]: [brand_id#27, class_id#28, category_id#29] (36) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (37) CometFilter -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] -Condition : isnotnull(ws_item_sk#29) +Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Condition : isnotnull(ws_item_sk#30) -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +(38) ReusedExchange [Reuses operator id: 15] +Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -(39) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(39) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] +Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#32] -Join type: Inner -Join condition: None +(40) CometProject +Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -(41) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] -Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] +(41) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#37] -(42) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#36] +(42) CometBroadcastHashJoin +Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] +Right output [1]: [d_date_sk#37] +Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None +(43) CometProject +Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] +Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] -(44) Project [codegen id : 9] -Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] +(44) ColumnarToRow [codegen id : 2] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] (45) BroadcastExchange -Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] +Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] -Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +(46) BroadcastHashJoin [codegen id : 3] +Left keys [6]: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)] +Right keys [6]: [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)] Join type: LeftSemi Join condition: None (47) BroadcastExchange -Input [3]: [brand_id#26, class_id#27, category_id#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] +Input [3]: [brand_id#27, class_id#28, category_id#29] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=3] -(48) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 4] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Right keys [3]: [brand_id#27, class_id#28, category_id#29] Join type: Inner Join condition: None -(49) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#37] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] +(49) Project [codegen id : 4] +Output [1]: [i_item_sk#6 AS ss_item_sk#38] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] (50) BroadcastExchange -Input [1]: [ss_item_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Input [1]: [ss_item_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(51) BroadcastHashJoin [codegen id : 25] +(51) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#37] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (52) Scan parquet spark_catalog.default.item -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (53) CometFilter -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Condition : isnotnull(i_item_sk#38) +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Condition : isnotnull(i_item_sk#39) -(54) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(54) ColumnarToRow [codegen id : 9] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] (55) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(56) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#38] -Right keys [1]: [ss_item_sk#37] +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#39] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (57) BroadcastExchange -Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -(58) BroadcastHashJoin [codegen id : 25] +(58) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#38] +Right keys [1]: [i_item_sk#39] Join type: Inner Join condition: None -(59) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(59) Project [codegen id : 11] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -(60) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#42] +(60) ReusedExchange [Reuses operator id: 155] +Output [1]: [d_date_sk#43] -(61) BroadcastHashJoin [codegen id : 25] +(61) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#42] +Right keys [1]: [d_date_sk#43] Join type: Inner Join condition: None -(62) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] +(62) Project [codegen id : 11] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -(63) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(63) HashAggregate [codegen id : 11] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] -Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Aggregate Attributes [3]: [sum#44, isEmpty#45, count#46] +Results [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] (64) Exchange -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(65) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +(65) HashAggregate [codegen id : 12] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] +Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] -Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50, count(1)#51] +Results [6]: [store AS channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#50 AS sales#53, count(1)#51 AS number_sales#54] -(66) Filter [codegen id : 26] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(66) Filter [codegen id : 12] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Condition : (isnotnull(sales#53) AND (cast(sales#53 as decimal(32,6)) > cast(Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (67) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#59), dynamicpruningexpression(cs_sold_date_sk#59 IN dynamicpruning#60)] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (68) CometFilter -Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] -Condition : isnotnull(cs_item_sk#56) +Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] +Condition : isnotnull(cs_item_sk#57) -(69) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +(69) ColumnarToRow [codegen id : 23] +Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] (70) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(71) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#56] -Right keys [1]: [ss_item_sk#37] +(71) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#57] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (72) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(73) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#56] -Right keys [1]: [i_item_sk#61] +(73) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#57] +Right keys [1]: [i_item_sk#62] Join type: Inner Join condition: None -(74) Project [codegen id : 51] -Output [6]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [8]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] +(74) Project [codegen id : 23] +Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(75) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#65] +(75) ReusedExchange [Reuses operator id: 155] +Output [1]: [d_date_sk#66] -(76) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#59] -Right keys [1]: [d_date_sk#65] +(76) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_sold_date_sk#60] +Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(77) Project [codegen id : 51] -Output [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [7]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] +(77) Project [codegen id : 23] +Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] -(78) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), partial_count(1)] -Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] -Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] +(78) HashAggregate [codegen id : 23] +Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] (79) Exchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(80) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#72, count(1)#73] -Results [6]: [catalog AS channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#72 AS sales#75, count(1)#73 AS number_sales#76] +(80) HashAggregate [codegen id : 24] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73, count(1)#74] +Results [6]: [catalog AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73 AS sales#76, count(1)#74 AS number_sales#77] -(81) Filter [codegen id : 52] -Input [6]: [channel#74, i_brand_id#62, i_class_id#63, i_category_id#64, sales#75, number_sales#76] -Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(81) Filter [codegen id : 24] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (82) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] +Output [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#81)] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (83) CometFilter -Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] -Condition : isnotnull(ws_item_sk#77) +Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_item_sk#78) -(84) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] +(84) ColumnarToRow [codegen id : 35] +Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] (85) ReusedExchange [Reuses operator id: 50] -Output [1]: [ss_item_sk#37] +Output [1]: [ss_item_sk#38] -(86) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#77] -Right keys [1]: [ss_item_sk#37] +(86) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#78] +Right keys [1]: [ss_item_sk#38] Join type: LeftSemi Join condition: None (87) ReusedExchange [Reuses operator id: 57] -Output [4]: [i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] +Output [4]: [i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] -(88) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#77] -Right keys [1]: [i_item_sk#82] +(88) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_item_sk#78] +Right keys [1]: [i_item_sk#83] Join type: Inner Join condition: None -(89) Project [codegen id : 77] -Output [6]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#83, i_class_id#84, i_category_id#85] -Input [8]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_item_sk#82, i_brand_id#83, i_class_id#84, i_category_id#85] +(89) Project [codegen id : 35] +Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86] +Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] -(90) ReusedExchange [Reuses operator id: 154] -Output [1]: [d_date_sk#86] +(90) ReusedExchange [Reuses operator id: 155] +Output [1]: [d_date_sk#87] -(91) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#86] +(91) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#87] Join type: Inner Join condition: None -(92) Project [codegen id : 77] -Output [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] -Input [7]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#83, i_class_id#84, i_category_id#85, d_date_sk#86] +(92) Project [codegen id : 35] +Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] +Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86, d_date_sk#87] -(93) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#83, i_class_id#84, i_category_id#85] -Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] -Functions [2]: [partial_sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), partial_count(1)] -Aggregate Attributes [3]: [sum#87, isEmpty#88, count#89] -Results [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] +(93) HashAggregate [codegen id : 35] +Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] +Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] +Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] +Aggregate Attributes [3]: [sum#88, isEmpty#89, count#90] +Results [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] (94) Exchange -Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] -Arguments: hashpartitioning(i_brand_id#83, i_class_id#84, i_category_id#85, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] +Arguments: hashpartitioning(i_brand_id#84, i_class_id#85, i_category_id#86, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(95) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#83, i_class_id#84, i_category_id#85, sum#90, isEmpty#91, count#92] -Keys [3]: [i_brand_id#83, i_class_id#84, i_category_id#85] -Functions [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#93, count(1)#94] -Results [6]: [web AS channel#95, i_brand_id#83, i_class_id#84, i_category_id#85, sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#93 AS sales#96, count(1)#94 AS number_sales#97] +(95) HashAggregate [codegen id : 36] +Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] +Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] +Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94, count(1)#95] +Results [6]: [web AS channel#96, i_brand_id#84, i_class_id#85, i_category_id#86, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94 AS sales#97, count(1)#95 AS number_sales#98] -(96) Filter [codegen id : 78] -Input [6]: [channel#95, i_brand_id#83, i_class_id#84, i_category_id#85, sales#96, number_sales#97] -Condition : (isnotnull(sales#96) AND (cast(sales#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) +(96) Filter [codegen id : 36] +Input [6]: [channel#96, i_brand_id#84, i_class_id#85, i_category_id#86, sales#97, number_sales#98] +Condition : (isnotnull(sales#97) AND (cast(sales#97 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#55, [id=#56] as decimal(32,6)))) (97) Union -(98) HashAggregate [codegen id : 79] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] -Aggregate Attributes [3]: [sum#98, isEmpty#99, sum#100] -Results [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] +(98) HashAggregate [codegen id : 37] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sales#53, number_sales#54] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [partial_sum(sales#53), partial_sum(number_sales#54)] +Aggregate Attributes [3]: [sum#99, isEmpty#100, sum#101] +Results [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] (99) Exchange -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Arguments: hashpartitioning(channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(100) HashAggregate [codegen id : 80] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] +(100) HashAggregate [codegen id : 38] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#53), sum(number_sales#54)] +Aggregate Attributes [2]: [sum(sales#53)#105, sum(number_sales#54)#106] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum(sales#53)#105 AS sum_sales#107, sum(number_sales#54)#106 AS number_sales#108] (101) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(102) HashAggregate [codegen id : 160] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] -Results [5]: [channel#51, i_brand_id#39, i_class_id#40, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] - -(103) HashAggregate [codegen id : 160] -Input [5]: [channel#51, i_brand_id#39, i_class_id#40, sum_sales#106, number_sales#107] -Keys [3]: [channel#51, i_brand_id#39, i_class_id#40] -Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] -Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] +Output [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] + +(102) HashAggregate [codegen id : 76] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#53), sum(number_sales#54)] +Aggregate Attributes [2]: [sum(sales#53)#105, sum(number_sales#54)#106] +Results [5]: [channel#52, i_brand_id#40, i_class_id#41, sum(sales#53)#105 AS sum_sales#107, sum(number_sales#54)#106 AS number_sales#108] + +(103) HashAggregate [codegen id : 76] +Input [5]: [channel#52, i_brand_id#40, i_class_id#41, sum_sales#107, number_sales#108] +Keys [3]: [channel#52, i_brand_id#40, i_class_id#41] +Functions [2]: [partial_sum(sum_sales#107), partial_sum(number_sales#108)] +Aggregate Attributes [3]: [sum#109, isEmpty#110, sum#111] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, sum#112, isEmpty#113, sum#114] (104) Exchange -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, sum#112, isEmpty#113, sum#114] +Arguments: hashpartitioning(channel#52, i_brand_id#40, i_class_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(105) HashAggregate [codegen id : 161] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, sum#111, isEmpty#112, sum#113] -Keys [3]: [channel#51, i_brand_id#39, i_class_id#40] -Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] -Aggregate Attributes [2]: [sum(sum_sales#106)#114, sum(number_sales#107)#115] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, null AS i_category_id#116, sum(sum_sales#106)#114 AS sum(sum_sales)#117, sum(number_sales#107)#115 AS sum(number_sales)#118] +(105) HashAggregate [codegen id : 77] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, sum#112, isEmpty#113, sum#114] +Keys [3]: [channel#52, i_brand_id#40, i_class_id#41] +Functions [2]: [sum(sum_sales#107), sum(number_sales#108)] +Aggregate Attributes [2]: [sum(sum_sales#107)#115, sum(number_sales#108)#116] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, null AS i_category_id#117, sum(sum_sales#107)#115 AS sum(sum_sales)#118, sum(number_sales#108)#116 AS sum(number_sales)#119] (106) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(107) HashAggregate [codegen id : 241] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] -Results [4]: [channel#51, i_brand_id#39, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] - -(108) HashAggregate [codegen id : 241] -Input [4]: [channel#51, i_brand_id#39, sum_sales#106, number_sales#107] -Keys [2]: [channel#51, i_brand_id#39] -Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] -Aggregate Attributes [3]: [sum#119, isEmpty#120, sum#121] -Results [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] +Output [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] + +(107) HashAggregate [codegen id : 115] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#53), sum(number_sales#54)] +Aggregate Attributes [2]: [sum(sales#53)#105, sum(number_sales#54)#106] +Results [4]: [channel#52, i_brand_id#40, sum(sales#53)#105 AS sum_sales#107, sum(number_sales#54)#106 AS number_sales#108] + +(108) HashAggregate [codegen id : 115] +Input [4]: [channel#52, i_brand_id#40, sum_sales#107, number_sales#108] +Keys [2]: [channel#52, i_brand_id#40] +Functions [2]: [partial_sum(sum_sales#107), partial_sum(number_sales#108)] +Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] +Results [5]: [channel#52, i_brand_id#40, sum#123, isEmpty#124, sum#125] (109) Exchange -Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] -Arguments: hashpartitioning(channel#51, i_brand_id#39, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Input [5]: [channel#52, i_brand_id#40, sum#123, isEmpty#124, sum#125] +Arguments: hashpartitioning(channel#52, i_brand_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(110) HashAggregate [codegen id : 242] -Input [5]: [channel#51, i_brand_id#39, sum#122, isEmpty#123, sum#124] -Keys [2]: [channel#51, i_brand_id#39] -Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] -Aggregate Attributes [2]: [sum(sum_sales#106)#125, sum(number_sales#107)#126] -Results [6]: [channel#51, i_brand_id#39, null AS i_class_id#127, null AS i_category_id#128, sum(sum_sales#106)#125 AS sum(sum_sales)#129, sum(number_sales#107)#126 AS sum(number_sales)#130] +(110) HashAggregate [codegen id : 116] +Input [5]: [channel#52, i_brand_id#40, sum#123, isEmpty#124, sum#125] +Keys [2]: [channel#52, i_brand_id#40] +Functions [2]: [sum(sum_sales#107), sum(number_sales#108)] +Aggregate Attributes [2]: [sum(sum_sales#107)#126, sum(number_sales#108)#127] +Results [6]: [channel#52, i_brand_id#40, null AS i_class_id#128, null AS i_category_id#129, sum(sum_sales#107)#126 AS sum(sum_sales)#130, sum(number_sales#108)#127 AS sum(number_sales)#131] (111) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] - -(112) HashAggregate [codegen id : 322] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] -Results [3]: [channel#51, sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] - -(113) HashAggregate [codegen id : 322] -Input [3]: [channel#51, sum_sales#106, number_sales#107] -Keys [1]: [channel#51] -Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] -Aggregate Attributes [3]: [sum#131, isEmpty#132, sum#133] -Results [4]: [channel#51, sum#134, isEmpty#135, sum#136] +Output [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] + +(112) HashAggregate [codegen id : 154] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#53), sum(number_sales#54)] +Aggregate Attributes [2]: [sum(sales#53)#105, sum(number_sales#54)#106] +Results [3]: [channel#52, sum(sales#53)#105 AS sum_sales#107, sum(number_sales#54)#106 AS number_sales#108] + +(113) HashAggregate [codegen id : 154] +Input [3]: [channel#52, sum_sales#107, number_sales#108] +Keys [1]: [channel#52] +Functions [2]: [partial_sum(sum_sales#107), partial_sum(number_sales#108)] +Aggregate Attributes [3]: [sum#132, isEmpty#133, sum#134] +Results [4]: [channel#52, sum#135, isEmpty#136, sum#137] (114) Exchange -Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] -Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Input [4]: [channel#52, sum#135, isEmpty#136, sum#137] +Arguments: hashpartitioning(channel#52, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(115) HashAggregate [codegen id : 323] -Input [4]: [channel#51, sum#134, isEmpty#135, sum#136] -Keys [1]: [channel#51] -Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] -Aggregate Attributes [2]: [sum(sum_sales#106)#137, sum(number_sales#107)#138] -Results [6]: [channel#51, null AS i_brand_id#139, null AS i_class_id#140, null AS i_category_id#141, sum(sum_sales#106)#137 AS sum(sum_sales)#142, sum(number_sales#107)#138 AS sum(number_sales)#143] +(115) HashAggregate [codegen id : 155] +Input [4]: [channel#52, sum#135, isEmpty#136, sum#137] +Keys [1]: [channel#52] +Functions [2]: [sum(sum_sales#107), sum(number_sales#108)] +Aggregate Attributes [2]: [sum(sum_sales#107)#138, sum(number_sales#108)#139] +Results [6]: [channel#52, null AS i_brand_id#140, null AS i_class_id#141, null AS i_category_id#142, sum(sum_sales#107)#138 AS sum(sum_sales)#143, sum(number_sales#108)#139 AS sum(number_sales)#144] (116) ReusedExchange [Reuses operator id: 99] -Output [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] +Output [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] -(117) HashAggregate [codegen id : 403] -Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#101, isEmpty#102, sum#103] -Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] -Functions [2]: [sum(sales#52), sum(number_sales#53)] -Aggregate Attributes [2]: [sum(sales#52)#104, sum(number_sales#53)#105] -Results [2]: [sum(sales#52)#104 AS sum_sales#106, sum(number_sales#53)#105 AS number_sales#107] +(117) HashAggregate [codegen id : 193] +Input [7]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum#102, isEmpty#103, sum#104] +Keys [4]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42] +Functions [2]: [sum(sales#53), sum(number_sales#54)] +Aggregate Attributes [2]: [sum(sales#53)#105, sum(number_sales#54)#106] +Results [2]: [sum(sales#53)#105 AS sum_sales#107, sum(number_sales#54)#106 AS number_sales#108] -(118) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#106, number_sales#107] +(118) HashAggregate [codegen id : 193] +Input [2]: [sum_sales#107, number_sales#108] Keys: [] -Functions [2]: [partial_sum(sum_sales#106), partial_sum(number_sales#107)] -Aggregate Attributes [3]: [sum#144, isEmpty#145, sum#146] -Results [3]: [sum#147, isEmpty#148, sum#149] +Functions [2]: [partial_sum(sum_sales#107), partial_sum(number_sales#108)] +Aggregate Attributes [3]: [sum#145, isEmpty#146, sum#147] +Results [3]: [sum#148, isEmpty#149, sum#150] (119) Exchange -Input [3]: [sum#147, isEmpty#148, sum#149] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] +Input [3]: [sum#148, isEmpty#149, sum#150] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] -(120) HashAggregate [codegen id : 404] -Input [3]: [sum#147, isEmpty#148, sum#149] +(120) HashAggregate [codegen id : 194] +Input [3]: [sum#148, isEmpty#149, sum#150] Keys: [] -Functions [2]: [sum(sum_sales#106), sum(number_sales#107)] -Aggregate Attributes [2]: [sum(sum_sales#106)#150, sum(number_sales#107)#151] -Results [6]: [null AS channel#152, null AS i_brand_id#153, null AS i_class_id#154, null AS i_category_id#155, sum(sum_sales#106)#150 AS sum(sum_sales)#156, sum(number_sales#107)#151 AS sum(number_sales)#157] +Functions [2]: [sum(sum_sales#107), sum(number_sales#108)] +Aggregate Attributes [2]: [sum(sum_sales#107)#151, sum(number_sales#108)#152] +Results [6]: [null AS channel#153, null AS i_brand_id#154, null AS i_class_id#155, null AS i_category_id#156, sum(sum_sales#107)#151 AS sum(sum_sales)#157, sum(number_sales#108)#152 AS sum(number_sales)#158] (121) Union -(122) HashAggregate [codegen id : 405] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +(122) HashAggregate [codegen id : 195] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] +Keys [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] (123) Exchange -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107, 5), ENSURE_REQUIREMENTS, [plan_id=17] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] +Arguments: hashpartitioning(channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(124) HashAggregate [codegen id : 406] -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +(124) HashAggregate [codegen id : 196] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] +Keys [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +Results [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] (125) TakeOrderedAndProject -Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] -Arguments: 100, [channel#51 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#106, number_sales#107] +Input [6]: [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] +Arguments: 100, [channel#52 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#52, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#107, number_sales#108] ===== Subqueries ===== -Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] -* HashAggregate (144) -+- Exchange (143) - +- * HashAggregate (142) - +- Union (141) - :- * Project (130) - : +- * BroadcastHashJoin Inner BuildRight (129) - : :- * ColumnarToRow (127) - : : +- CometScan parquet spark_catalog.default.store_sales (126) - : +- ReusedExchange (128) - :- * Project (135) - : +- * BroadcastHashJoin Inner BuildRight (134) - : :- * ColumnarToRow (132) - : : +- CometScan parquet spark_catalog.default.catalog_sales (131) - : +- ReusedExchange (133) - +- * Project (140) - +- * BroadcastHashJoin Inner BuildRight (139) - :- * ColumnarToRow (137) - : +- CometScan parquet spark_catalog.default.web_sales (136) - +- ReusedExchange (138) +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#55, [id=#56] +* HashAggregate (145) ++- Exchange (144) + +- * ColumnarToRow (143) + +- CometHashAggregate (142) + +- CometUnion (141) + :- CometProject (129) + : +- CometBroadcastHashJoin (128) + : :- CometScan parquet spark_catalog.default.store_sales (126) + : +- ReusedExchange (127) + :- CometProject (136) + : +- CometBroadcastHashJoin (135) + : :- CometScan parquet spark_catalog.default.catalog_sales (130) + : +- CometBroadcastExchange (134) + : +- CometProject (133) + : +- CometFilter (132) + : +- CometScan parquet spark_catalog.default.date_dim (131) + +- CometProject (140) + +- CometBroadcastHashJoin (139) + :- CometScan parquet spark_catalog.default.web_sales (137) + +- ReusedExchange (138) (126) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#158, ss_list_price#159, ss_sold_date_sk#160] +Output [3]: [ss_quantity#159, ss_list_price#160, ss_sold_date_sk#161] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#160), dynamicpruningexpression(ss_sold_date_sk#160 IN dynamicpruning#161)] +PartitionFilters: [isnotnull(ss_sold_date_sk#161), dynamicpruningexpression(ss_sold_date_sk#161 IN dynamicpruning#162)] ReadSchema: struct -(127) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#158, ss_list_price#159, ss_sold_date_sk#160] - -(128) ReusedExchange [Reuses operator id: 159] -Output [1]: [d_date_sk#162] +(127) ReusedExchange [Reuses operator id: 21] +Output [1]: [d_date_sk#163] -(129) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#160] -Right keys [1]: [d_date_sk#162] -Join type: Inner -Join condition: None +(128) CometBroadcastHashJoin +Left output [3]: [ss_quantity#159, ss_list_price#160, ss_sold_date_sk#161] +Right output [1]: [d_date_sk#163] +Arguments: [ss_sold_date_sk#161], [d_date_sk#163], Inner, BuildRight -(130) Project [codegen id : 2] -Output [2]: [ss_quantity#158 AS quantity#163, ss_list_price#159 AS list_price#164] -Input [4]: [ss_quantity#158, ss_list_price#159, ss_sold_date_sk#160, d_date_sk#162] +(129) CometProject +Input [4]: [ss_quantity#159, ss_list_price#160, ss_sold_date_sk#161, d_date_sk#163] +Arguments: [quantity#164, list_price#165], [ss_quantity#159 AS quantity#164, ss_list_price#160 AS list_price#165] -(131) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#165, cs_list_price#166, cs_sold_date_sk#167] +(130) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#166, cs_list_price#167, cs_sold_date_sk#168] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#167), dynamicpruningexpression(cs_sold_date_sk#167 IN dynamicpruning#168)] +PartitionFilters: [isnotnull(cs_sold_date_sk#168), dynamicpruningexpression(cs_sold_date_sk#168 IN dynamicpruning#169)] ReadSchema: struct -(132) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#165, cs_list_price#166, cs_sold_date_sk#167] +(131) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#170, d_year#171] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(133) ReusedExchange [Reuses operator id: 149] -Output [1]: [d_date_sk#169] +(132) CometFilter +Input [2]: [d_date_sk#170, d_year#171] +Condition : (((isnotnull(d_year#171) AND (d_year#171 >= 1998)) AND (d_year#171 <= 2000)) AND isnotnull(d_date_sk#170)) -(134) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#167] -Right keys [1]: [d_date_sk#169] -Join type: Inner -Join condition: None +(133) CometProject +Input [2]: [d_date_sk#170, d_year#171] +Arguments: [d_date_sk#170], [d_date_sk#170] + +(134) CometBroadcastExchange +Input [1]: [d_date_sk#170] +Arguments: [d_date_sk#170] -(135) Project [codegen id : 4] -Output [2]: [cs_quantity#165 AS quantity#170, cs_list_price#166 AS list_price#171] -Input [4]: [cs_quantity#165, cs_list_price#166, cs_sold_date_sk#167, d_date_sk#169] +(135) CometBroadcastHashJoin +Left output [3]: [cs_quantity#166, cs_list_price#167, cs_sold_date_sk#168] +Right output [1]: [d_date_sk#170] +Arguments: [cs_sold_date_sk#168], [d_date_sk#170], Inner, BuildRight -(136) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#172, ws_list_price#173, ws_sold_date_sk#174] +(136) CometProject +Input [4]: [cs_quantity#166, cs_list_price#167, cs_sold_date_sk#168, d_date_sk#170] +Arguments: [quantity#172, list_price#173], [cs_quantity#166 AS quantity#172, cs_list_price#167 AS list_price#173] + +(137) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#174, ws_list_price#175, ws_sold_date_sk#176] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#174), dynamicpruningexpression(ws_sold_date_sk#174 IN dynamicpruning#175)] +PartitionFilters: [isnotnull(ws_sold_date_sk#176), dynamicpruningexpression(ws_sold_date_sk#176 IN dynamicpruning#177)] ReadSchema: struct -(137) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#172, ws_list_price#173, ws_sold_date_sk#174] - -(138) ReusedExchange [Reuses operator id: 149] -Output [1]: [d_date_sk#176] +(138) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#178] -(139) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#174] -Right keys [1]: [d_date_sk#176] -Join type: Inner -Join condition: None +(139) CometBroadcastHashJoin +Left output [3]: [ws_quantity#174, ws_list_price#175, ws_sold_date_sk#176] +Right output [1]: [d_date_sk#178] +Arguments: [ws_sold_date_sk#176], [d_date_sk#178], Inner, BuildRight -(140) Project [codegen id : 6] -Output [2]: [ws_quantity#172 AS quantity#177, ws_list_price#173 AS list_price#178] -Input [4]: [ws_quantity#172, ws_list_price#173, ws_sold_date_sk#174, d_date_sk#176] +(140) CometProject +Input [4]: [ws_quantity#174, ws_list_price#175, ws_sold_date_sk#176, d_date_sk#178] +Arguments: [quantity#179, list_price#180], [ws_quantity#174 AS quantity#179, ws_list_price#175 AS list_price#180] -(141) Union +(141) CometUnion +Child 0 Input [2]: [quantity#164, list_price#165] +Child 1 Input [2]: [quantity#172, list_price#173] +Child 2 Input [2]: [quantity#179, list_price#180] -(142) HashAggregate [codegen id : 7] -Input [2]: [quantity#163, list_price#164] +(142) CometHashAggregate +Input [2]: [quantity#164, list_price#165] Keys: [] -Functions [1]: [partial_avg((cast(quantity#163 as decimal(10,0)) * list_price#164))] -Aggregate Attributes [2]: [sum#179, count#180] -Results [2]: [sum#181, count#182] +Functions [1]: [partial_avg((cast(quantity#164 as decimal(10,0)) * list_price#165))] + +(143) ColumnarToRow [codegen id : 1] +Input [2]: [sum#181, count#182] -(143) Exchange +(144) Exchange Input [2]: [sum#181, count#182] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] -(144) HashAggregate [codegen id : 8] +(145) HashAggregate [codegen id : 2] Input [2]: [sum#181, count#182] Keys: [] -Functions [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))] -Aggregate Attributes [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))#183] -Results [1]: [avg((cast(quantity#163 as decimal(10,0)) * list_price#164))#183 AS average_sales#184] +Functions [1]: [avg((cast(quantity#164 as decimal(10,0)) * list_price#165))] +Aggregate Attributes [1]: [avg((cast(quantity#164 as decimal(10,0)) * list_price#165))#183] +Results [1]: [avg((cast(quantity#164 as decimal(10,0)) * list_price#165))#183 AS average_sales#184] -Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#160 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#161 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 131 Hosting Expression = cs_sold_date_sk#167 IN dynamicpruning#168 -BroadcastExchange (149) -+- * ColumnarToRow (148) - +- CometProject (147) - +- CometFilter (146) - +- CometScan parquet spark_catalog.default.date_dim (145) +Subquery:3 Hosting operator id = 130 Hosting Expression = cs_sold_date_sk#168 IN dynamicpruning#169 +BroadcastExchange (150) ++- * ColumnarToRow (149) + +- CometProject (148) + +- CometFilter (147) + +- CometScan parquet spark_catalog.default.date_dim (146) -(145) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#169, d_year#185] +(146) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#170, d_year#171] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(146) CometFilter -Input [2]: [d_date_sk#169, d_year#185] -Condition : (((isnotnull(d_year#185) AND (d_year#185 >= 1998)) AND (d_year#185 <= 2000)) AND isnotnull(d_date_sk#169)) +(147) CometFilter +Input [2]: [d_date_sk#170, d_year#171] +Condition : (((isnotnull(d_year#171) AND (d_year#171 >= 1998)) AND (d_year#171 <= 2000)) AND isnotnull(d_date_sk#170)) -(147) CometProject -Input [2]: [d_date_sk#169, d_year#185] -Arguments: [d_date_sk#169], [d_date_sk#169] +(148) CometProject +Input [2]: [d_date_sk#170, d_year#171] +Arguments: [d_date_sk#170], [d_date_sk#170] -(148) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#169] +(149) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#170] -(149) BroadcastExchange -Input [1]: [d_date_sk#169] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] +(150) BroadcastExchange +Input [1]: [d_date_sk#170] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] -Subquery:4 Hosting operator id = 136 Hosting Expression = ws_sold_date_sk#174 IN dynamicpruning#168 +Subquery:4 Hosting operator id = 137 Hosting Expression = ws_sold_date_sk#176 IN dynamicpruning#169 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (154) -+- * ColumnarToRow (153) - +- CometProject (152) - +- CometFilter (151) - +- CometScan parquet spark_catalog.default.date_dim (150) +BroadcastExchange (155) ++- * ColumnarToRow (154) + +- CometProject (153) + +- CometFilter (152) + +- CometScan parquet spark_catalog.default.date_dim (151) -(150) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#42, d_year#186, d_moy#187] +(151) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#43, d_year#185, d_moy#186] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(151) CometFilter -Input [3]: [d_date_sk#42, d_year#186, d_moy#187] -Condition : ((((isnotnull(d_year#186) AND isnotnull(d_moy#187)) AND (d_year#186 = 2000)) AND (d_moy#187 = 11)) AND isnotnull(d_date_sk#42)) +(152) CometFilter +Input [3]: [d_date_sk#43, d_year#185, d_moy#186] +Condition : ((((isnotnull(d_year#185) AND isnotnull(d_moy#186)) AND (d_year#185 = 2000)) AND (d_moy#186 = 11)) AND isnotnull(d_date_sk#43)) -(152) CometProject -Input [3]: [d_date_sk#42, d_year#186, d_moy#187] -Arguments: [d_date_sk#42], [d_date_sk#42] +(153) CometProject +Input [3]: [d_date_sk#43, d_year#185, d_moy#186] +Arguments: [d_date_sk#43], [d_date_sk#43] -(153) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#42] +(154) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#43] -(154) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] +(155) BroadcastExchange +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (159) -+- * ColumnarToRow (158) - +- CometProject (157) - +- CometFilter (156) - +- CometScan parquet spark_catalog.default.date_dim (155) +BroadcastExchange (160) ++- * ColumnarToRow (159) + +- CometProject (158) + +- CometFilter (157) + +- CometScan parquet spark_catalog.default.date_dim (156) -(155) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#188] +(156) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_year#187] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(156) CometFilter -Input [2]: [d_date_sk#25, d_year#188] -Condition : (((isnotnull(d_year#188) AND (d_year#188 >= 1999)) AND (d_year#188 <= 2001)) AND isnotnull(d_date_sk#25)) +(157) CometFilter +Input [2]: [d_date_sk#26, d_year#187] +Condition : (((isnotnull(d_year#187) AND (d_year#187 >= 1999)) AND (d_year#187 <= 2001)) AND isnotnull(d_date_sk#26)) -(157) CometProject -Input [2]: [d_date_sk#25, d_year#188] -Arguments: [d_date_sk#25], [d_date_sk#25] +(158) CometProject +Input [2]: [d_date_sk#26, d_year#187] +Arguments: [d_date_sk#26], [d_date_sk#26] -(158) ColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] +(159) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#26] -(159) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] +(160) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=18] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] +Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#55, [id=#56] -Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index a203f9620b..45061c290c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -1,69 +1,61 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (406) + WholeStageCodegen (196) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) + WholeStageCodegen (195) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (80) + WholeStageCodegen (38) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) + WholeStageCodegen (37) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (26) + WholeStageCodegen (12) Filter [sales] Subquery #3 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] InputAdapter - Exchange #14 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] + Exchange #15 + WholeStageCodegen (1) + ColumnarToRow InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #15 + CometHashAggregate [quantity,list_price] + CometUnion + CometProject [ss_quantity,ss_list_price] [quantity,list_price] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk] #12 + CometProject [cs_quantity,cs_list_price] [quantity,list_price] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #16 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #17 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [ws_quantity,ws_list_price] [quantity,list_price] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + ReusedExchange [d_date_sk] #17 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) + WholeStageCodegen (11) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -84,7 +76,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 - WholeStageCodegen (11) + WholeStageCodegen (4) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] ColumnarToRow @@ -93,19 +85,19 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 - WholeStageCodegen (10) + WholeStageCodegen (3) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [brand_id,class_id,category_id] + CometProject [i_brand_id,i_class_id,i_category_id] [brand_id,class_id,category_id] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 @@ -116,56 +108,43 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 + CometBroadcastExchange #9 + CometBroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #10 + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + CometBroadcastExchange #11 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastExchange #12 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk] #12 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #13 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + ReusedExchange [d_date_sk] #12 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) + BroadcastExchange #14 + WholeStageCodegen (9) BroadcastHashJoin [i_item_sk,ss_item_sk] ColumnarToRow InputAdapter @@ -175,13 +154,13 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [ss_item_sk] #5 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) + WholeStageCodegen (24) Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) + Exchange [i_brand_id,i_class_id,i_category_id] #18 + WholeStageCodegen (23) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -196,16 +175,16 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) + WholeStageCodegen (36) Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) + Exchange [i_brand_id,i_class_id,i_category_id] #19 + WholeStageCodegen (35) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -220,41 +199,41 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) + WholeStageCodegen (77) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) + Exchange [channel,i_brand_id,i_class_id] #20 + WholeStageCodegen (76) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) + WholeStageCodegen (116) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel,i_brand_id] #19 - WholeStageCodegen (241) + Exchange [channel,i_brand_id] #21 + WholeStageCodegen (115) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) + WholeStageCodegen (155) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange [channel] #20 - WholeStageCodegen (322) + Exchange [channel] #22 + WholeStageCodegen (154) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) + WholeStageCodegen (194) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter - Exchange #21 - WholeStageCodegen (403) + Exchange #23 + WholeStageCodegen (193) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt index 6bd6459861..1ad55cbea2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt @@ -1,157 +1,151 @@ == Physical Plan == -TakeOrderedAndProject (153) -+- Union (152) - :- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (32) - : : +- * BroadcastHashJoin Inner BuildRight (31) - : : :- * Project (29) - : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * ColumnarToRow (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * ColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * ColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometScan parquet spark_catalog.default.customer (11) - : : : : +- BroadcastExchange (21) - : : : : +- * ColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) - : : : +- BroadcastExchange (27) - : : : +- * ColumnarToRow (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.customer_address (24) - : : +- ReusedExchange (30) - : +- BroadcastExchange (36) - : +- * ColumnarToRow (35) - : +- CometFilter (34) - : +- CometScan parquet spark_catalog.default.item (33) - :- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * Project (53) - : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : :- * Project (50) - : : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : : :- * Project (47) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : : : :- * ColumnarToRow (44) - : : : : : : : +- CometFilter (43) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (42) - : : : : : : +- ReusedExchange (45) - : : : : : +- ReusedExchange (48) - : : : : +- ReusedExchange (51) - : : : +- BroadcastExchange (57) - : : : +- * ColumnarToRow (56) - : : : +- CometFilter (55) - : : : +- CometScan parquet spark_catalog.default.customer_address (54) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - :- * HashAggregate (96) - : +- Exchange (95) - : +- * HashAggregate (94) - : +- * Project (93) - : +- * BroadcastHashJoin Inner BuildRight (92) - : :- * Project (90) - : : +- * BroadcastHashJoin Inner BuildRight (89) - : : :- * Project (87) - : : : +- * BroadcastHashJoin Inner BuildRight (86) - : : : :- * Project (80) - : : : : +- * BroadcastHashJoin Inner BuildRight (79) - : : : : :- * Project (77) - : : : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : : : :- * Project (74) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : : : : :- * ColumnarToRow (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (69) - : : : : : : +- ReusedExchange (72) - : : : : : +- ReusedExchange (75) - : : : : +- ReusedExchange (78) - : : : +- BroadcastExchange (85) - : : : +- * ColumnarToRow (84) - : : : +- CometProject (83) - : : : +- CometFilter (82) - : : : +- CometScan parquet spark_catalog.default.customer_address (81) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- * HashAggregate (124) - : +- Exchange (123) - : +- * HashAggregate (122) - : +- * Project (121) - : +- * BroadcastHashJoin Inner BuildRight (120) - : :- * Project (118) - : : +- * BroadcastHashJoin Inner BuildRight (117) - : : :- * Project (115) - : : : +- * BroadcastHashJoin Inner BuildRight (114) - : : : :- * Project (108) - : : : : +- * BroadcastHashJoin Inner BuildRight (107) - : : : : :- * Project (105) - : : : : : +- * BroadcastHashJoin Inner BuildRight (104) - : : : : : :- * Project (102) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (101) - : : : : : : :- * ColumnarToRow (99) - : : : : : : : +- CometFilter (98) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (97) - : : : : : : +- ReusedExchange (100) - : : : : : +- ReusedExchange (103) - : : : : +- ReusedExchange (106) - : : : +- BroadcastExchange (113) - : : : +- * ColumnarToRow (112) - : : : +- CometProject (111) - : : : +- CometFilter (110) - : : : +- CometScan parquet spark_catalog.default.customer_address (109) - : : +- ReusedExchange (116) - : +- ReusedExchange (119) - +- * HashAggregate (151) - +- Exchange (150) - +- * HashAggregate (149) - +- * Project (148) - +- * BroadcastHashJoin Inner BuildRight (147) - :- * Project (142) - : +- * BroadcastHashJoin Inner BuildRight (141) - : :- * Project (139) - : : +- * BroadcastHashJoin Inner BuildRight (138) - : : :- * Project (136) - : : : +- * BroadcastHashJoin Inner BuildRight (135) - : : : :- * Project (133) - : : : : +- * BroadcastHashJoin Inner BuildRight (132) - : : : : :- * Project (130) - : : : : : +- * BroadcastHashJoin Inner BuildRight (129) - : : : : : :- * ColumnarToRow (127) - : : : : : : +- CometFilter (126) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (125) - : : : : : +- ReusedExchange (128) - : : : : +- ReusedExchange (131) - : : : +- ReusedExchange (134) - : : +- ReusedExchange (137) - : +- ReusedExchange (140) - +- BroadcastExchange (146) - +- * ColumnarToRow (145) - +- CometFilter (144) - +- CometScan parquet spark_catalog.default.item (143) +TakeOrderedAndProject (147) ++- Union (146) + :- * HashAggregate (39) + : +- Exchange (38) + : +- * ColumnarToRow (37) + : +- CometHashAggregate (36) + : +- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometProject (19) + : : : : +- CometBroadcastHashJoin (18) + : : : : :- CometProject (14) + : : : : : +- CometBroadcastHashJoin (13) + : : : : : :- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : +- CometBroadcastExchange (6) + : : : : : : +- CometProject (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : : : : +- CometBroadcastExchange (12) + : : : : : +- CometProject (11) + : : : : : +- CometFilter (10) + : : : : : +- CometScan parquet spark_catalog.default.customer (9) + : : : : +- CometBroadcastExchange (17) + : : : : +- CometFilter (16) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (15) + : : : +- CometBroadcastExchange (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.customer_address (20) + : : +- CometBroadcastExchange (28) + : : +- CometProject (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- CometBroadcastExchange (33) + : +- CometFilter (32) + : +- CometScan parquet spark_catalog.default.item (31) + :- * HashAggregate (65) + : +- Exchange (64) + : +- * ColumnarToRow (63) + : +- CometHashAggregate (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (58) + : : +- CometBroadcastHashJoin (57) + : : :- CometProject (55) + : : : +- CometBroadcastHashJoin (54) + : : : :- CometProject (50) + : : : : +- CometBroadcastHashJoin (49) + : : : : :- CometProject (47) + : : : : : +- CometBroadcastHashJoin (46) + : : : : : :- CometProject (44) + : : : : : : +- CometBroadcastHashJoin (43) + : : : : : : :- CometFilter (41) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) + : : : : : : +- ReusedExchange (42) + : : : : : +- ReusedExchange (45) + : : : : +- ReusedExchange (48) + : : : +- CometBroadcastExchange (53) + : : : +- CometFilter (52) + : : : +- CometScan parquet spark_catalog.default.customer_address (51) + : : +- ReusedExchange (56) + : +- ReusedExchange (59) + :- * HashAggregate (92) + : +- Exchange (91) + : +- * ColumnarToRow (90) + : +- CometHashAggregate (89) + : +- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometProject (85) + : : +- CometBroadcastHashJoin (84) + : : :- CometProject (82) + : : : +- CometBroadcastHashJoin (81) + : : : :- CometProject (76) + : : : : +- CometBroadcastHashJoin (75) + : : : : :- CometProject (73) + : : : : : +- CometBroadcastHashJoin (72) + : : : : : :- CometProject (70) + : : : : : : +- CometBroadcastHashJoin (69) + : : : : : : :- CometFilter (67) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (66) + : : : : : : +- ReusedExchange (68) + : : : : : +- ReusedExchange (71) + : : : : +- ReusedExchange (74) + : : : +- CometBroadcastExchange (80) + : : : +- CometProject (79) + : : : +- CometFilter (78) + : : : +- CometScan parquet spark_catalog.default.customer_address (77) + : : +- ReusedExchange (83) + : +- ReusedExchange (86) + :- * HashAggregate (119) + : +- Exchange (118) + : +- * ColumnarToRow (117) + : +- CometHashAggregate (116) + : +- CometProject (115) + : +- CometBroadcastHashJoin (114) + : :- CometProject (112) + : : +- CometBroadcastHashJoin (111) + : : :- CometProject (109) + : : : +- CometBroadcastHashJoin (108) + : : : :- CometProject (103) + : : : : +- CometBroadcastHashJoin (102) + : : : : :- CometProject (100) + : : : : : +- CometBroadcastHashJoin (99) + : : : : : :- CometProject (97) + : : : : : : +- CometBroadcastHashJoin (96) + : : : : : : :- CometFilter (94) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (93) + : : : : : : +- ReusedExchange (95) + : : : : : +- ReusedExchange (98) + : : : : +- ReusedExchange (101) + : : : +- CometBroadcastExchange (107) + : : : +- CometProject (106) + : : : +- CometFilter (105) + : : : +- CometScan parquet spark_catalog.default.customer_address (104) + : : +- ReusedExchange (110) + : +- ReusedExchange (113) + +- * HashAggregate (145) + +- Exchange (144) + +- * ColumnarToRow (143) + +- CometHashAggregate (142) + +- CometProject (141) + +- CometBroadcastHashJoin (140) + :- CometProject (136) + : +- CometBroadcastHashJoin (135) + : :- CometProject (133) + : : +- CometBroadcastHashJoin (132) + : : :- CometProject (130) + : : : +- CometBroadcastHashJoin (129) + : : : :- CometProject (127) + : : : : +- CometBroadcastHashJoin (126) + : : : : :- CometProject (124) + : : : : : +- CometBroadcastHashJoin (123) + : : : : : :- CometFilter (121) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (120) + : : : : : +- ReusedExchange (122) + : : : : +- ReusedExchange (125) + : : : +- ReusedExchange (128) + : : +- ReusedExchange (131) + : +- ReusedExchange (134) + +- CometBroadcastExchange (139) + +- CometFilter (138) + +- CometScan parquet spark_catalog.default.item (137) (1) Scan parquet spark_catalog.default.catalog_sales @@ -166,744 +160,693 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = M)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#11)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) ColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cd_demo_sk#11, cd_dep_count#14] -(8) BroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Right output [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +(8) CometProject Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(11) Scan parquet spark_catalog.default.customer +(9) Scan parquet spark_catalog.default.customer Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(12) CometFilter +(10) CometFilter Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) -(13) CometProject +(11) CometProject Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) ColumnarToRow [codegen id : 2] +(12) CometBroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(15) BroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(13) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +(14) CometProject Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(18) Scan parquet spark_catalog.default.customer_demographics +(15) Scan parquet spark_catalog.default.customer_demographics Output [1]: [cd_demo_sk#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(19) CometFilter +(16) CometFilter Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) ColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#20] - -(21) BroadcastExchange +(17) CometBroadcastExchange Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [cd_demo_sk#20] -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [cd_demo_sk#20] +Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +(19) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(24) Scan parquet spark_catalog.default.customer_address +(20) Scan parquet spark_catalog.default.customer_address Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(25) CometFilter +(21) CometFilter Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(26) ColumnarToRow [codegen id : 4] +(22) CometBroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -(27) BroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(23) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight -(28) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +(24) CometProject Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] -(30) ReusedExchange [Reuses operator id: 158] -Output [1]: [d_date_sk#25] +(25) Scan parquet spark_catalog.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,2001), IsNotNull(d_date_sk)] +ReadSchema: struct -(31) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(26) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#25)) + +(27) CometProject +Input [2]: [d_date_sk#25, d_year#26] +Arguments: [d_date_sk#25], [d_date_sk#25] -(32) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +(28) CometBroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: [d_date_sk#25] + +(29) CometBroadcastHashJoin +Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#9], [d_date_sk#25], Inner, BuildRight + +(30) CometProject Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, d_date_sk#25] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] -(33) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#26, i_item_id#27] +(31) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#27, i_item_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) CometFilter -Input [2]: [i_item_sk#26, i_item_id#27] -Condition : isnotnull(i_item_sk#26) - -(35) ColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#26, i_item_id#27] - -(36) BroadcastExchange -Input [2]: [i_item_sk#26, i_item_id#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 7] -Output [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#26, i_item_id#27] - -(39) HashAggregate [codegen id : 7] -Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] -Keys [4]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22] -Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Results [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] - -(40) Exchange -Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, ca_county#22, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(41) HashAggregate [codegen id : 8] -Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Keys [4]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22] -Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] -Aggregate Attributes [7]: [avg(agg1#28)#63, avg(agg2#29)#64, avg(agg3#30)#65, avg(agg4#31)#66, avg(agg5#32)#67, avg(agg6#33)#68, avg(agg7#34)#69] -Results [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, avg(agg1#28)#63 AS agg1#70, avg(agg2#29)#64 AS agg2#71, avg(agg3#30)#65 AS agg3#72, avg(agg4#31)#66 AS agg4#73, avg(agg5#32)#67 AS agg5#74, avg(agg6#33)#68 AS agg6#75, avg(agg7#34)#69 AS agg7#76] - -(42) Scan parquet spark_catalog.default.catalog_sales +(32) CometFilter +Input [2]: [i_item_sk#27, i_item_id#28] +Condition : isnotnull(i_item_sk#27) + +(33) CometBroadcastExchange +Input [2]: [i_item_sk#27, i_item_id#28] +Arguments: [i_item_sk#27, i_item_id#28] + +(34) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Right output [2]: [i_item_sk#27, i_item_id#28] +Arguments: [cs_item_sk#3], [i_item_sk#27], Inner, BuildRight + +(35) CometProject +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#27, i_item_id#28] +Arguments: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35], [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, cast(cs_quantity#4 as decimal(12,2)) AS agg1#29, cast(cs_list_price#5 as decimal(12,2)) AS agg2#30, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#31, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#32, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#33, cast(c_birth_year#19 as decimal(12,2)) AS agg6#34, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#35] + +(36) CometHashAggregate +Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35] +Keys [4]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22] +Functions [7]: [partial_avg(agg1#29), partial_avg(agg2#30), partial_avg(agg3#31), partial_avg(agg4#32), partial_avg(agg5#33), partial_avg(agg6#34), partial_avg(agg7#35)] + +(37) ColumnarToRow [codegen id : 1] +Input [18]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] + +(38) Exchange +Input [18]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] +Arguments: hashpartitioning(i_item_id#28, ca_country#24, ca_state#23, ca_county#22, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(39) HashAggregate [codegen id : 2] +Input [18]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] +Keys [4]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22] +Functions [7]: [avg(agg1#29), avg(agg2#30), avg(agg3#31), avg(agg4#32), avg(agg5#33), avg(agg6#34), avg(agg7#35)] +Aggregate Attributes [7]: [avg(agg1#29)#50, avg(agg2#30)#51, avg(agg3#31)#52, avg(agg4#32)#53, avg(agg5#33)#54, avg(agg6#34)#55, avg(agg7#35)#56] +Results [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, avg(agg1#29)#50 AS agg1#57, avg(agg2#30)#51 AS agg2#58, avg(agg3#31)#52 AS agg3#59, avg(agg4#32)#53 AS agg4#60, avg(agg5#33)#54 AS agg5#61, avg(agg6#34)#55 AS agg6#62, avg(agg7#35)#56 AS agg7#63] + +(40) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#64)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(43) CometFilter +(41) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(44) ColumnarToRow [codegen id : 15] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] - -(45) ReusedExchange [Reuses operator id: 8] +(42) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(46) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None +(43) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Right output [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight -(47) Project [codegen id : 15] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +(44) CometProject Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(48) ReusedExchange [Reuses operator id: 15] +(45) ReusedExchange [Reuses operator id: 12] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(49) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None +(46) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight -(50) Project [codegen id : 15] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +(47) CometProject Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(51) ReusedExchange [Reuses operator id: 21] +(48) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#20] -(52) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(49) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [cd_demo_sk#20] +Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight -(53) Project [codegen id : 15] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +(50) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(54) Scan parquet spark_catalog.default.customer_address +(51) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(55) CometFilter +(52) CometFilter Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(56) ColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] - -(57) BroadcastExchange +(53) CometBroadcastExchange Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +Arguments: [ca_address_sk#21, ca_state#23, ca_country#24] -(58) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +Right output [3]: [ca_address_sk#21, ca_state#23, ca_country#24] +Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight -(59) Project [codegen id : 15] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] +(55) CometProject Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_state#23, ca_country#24] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] -(60) ReusedExchange [Reuses operator id: 158] +(56) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#25] -(61) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(57) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#9], [d_date_sk#25], Inner, BuildRight -(62) Project [codegen id : 15] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] +(58) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24, d_date_sk#25] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] + +(59) ReusedExchange [Reuses operator id: 33] +Output [2]: [i_item_sk#27, i_item_id#28] -(63) ReusedExchange [Reuses operator id: 36] -Output [2]: [i_item_sk#26, i_item_id#27] - -(64) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 15] -Output [10]: [i_item_id#27, ca_country#24, ca_state#23, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24, i_item_sk#26, i_item_id#27] - -(66) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#27, ca_country#24, ca_state#23, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] -Keys [3]: [i_item_id#27, ca_country#24, ca_state#23] -Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -Results [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] - -(67) Exchange -Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] -Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(68) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#27, ca_country#24, ca_state#23, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103, sum#104, count#105] -Keys [3]: [i_item_id#27, ca_country#24, ca_state#23] -Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] -Aggregate Attributes [7]: [avg(agg1#28)#106, avg(agg2#29)#107, avg(agg3#30)#108, avg(agg4#31)#109, avg(agg5#32)#110, avg(agg6#33)#111, avg(agg7#34)#112] -Results [11]: [i_item_id#27, ca_country#24, ca_state#23, null AS county#113, avg(agg1#28)#106 AS agg1#114, avg(agg2#29)#107 AS agg2#115, avg(agg3#30)#108 AS agg3#116, avg(agg4#31)#109 AS agg4#117, avg(agg5#32)#110 AS agg5#118, avg(agg6#33)#111 AS agg6#119, avg(agg7#34)#112 AS agg7#120] - -(69) Scan parquet spark_catalog.default.catalog_sales +(60) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24] +Right output [2]: [i_item_sk#27, i_item_id#28] +Arguments: [cs_item_sk#3], [i_item_sk#27], Inner, BuildRight + +(61) CometProject +Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_state#23, ca_country#24, i_item_sk#27, i_item_id#28] +Arguments: [i_item_id#28, ca_country#24, ca_state#23, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35], [i_item_id#28, ca_country#24, ca_state#23, cast(cs_quantity#4 as decimal(12,2)) AS agg1#29, cast(cs_list_price#5 as decimal(12,2)) AS agg2#30, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#31, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#32, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#33, cast(c_birth_year#19 as decimal(12,2)) AS agg6#34, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#35] + +(62) CometHashAggregate +Input [10]: [i_item_id#28, ca_country#24, ca_state#23, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35] +Keys [3]: [i_item_id#28, ca_country#24, ca_state#23] +Functions [7]: [partial_avg(agg1#29), partial_avg(agg2#30), partial_avg(agg3#31), partial_avg(agg4#32), partial_avg(agg5#33), partial_avg(agg6#34), partial_avg(agg7#35)] + +(63) ColumnarToRow [codegen id : 3] +Input [17]: [i_item_id#28, ca_country#24, ca_state#23, sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76, sum#77, count#78] + +(64) Exchange +Input [17]: [i_item_id#28, ca_country#24, ca_state#23, sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76, sum#77, count#78] +Arguments: hashpartitioning(i_item_id#28, ca_country#24, ca_state#23, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(65) HashAggregate [codegen id : 4] +Input [17]: [i_item_id#28, ca_country#24, ca_state#23, sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76, sum#77, count#78] +Keys [3]: [i_item_id#28, ca_country#24, ca_state#23] +Functions [7]: [avg(agg1#29), avg(agg2#30), avg(agg3#31), avg(agg4#32), avg(agg5#33), avg(agg6#34), avg(agg7#35)] +Aggregate Attributes [7]: [avg(agg1#29)#79, avg(agg2#30)#80, avg(agg3#31)#81, avg(agg4#32)#82, avg(agg5#33)#83, avg(agg6#34)#84, avg(agg7#35)#85] +Results [11]: [i_item_id#28, ca_country#24, ca_state#23, null AS county#86, avg(agg1#29)#79 AS agg1#87, avg(agg2#30)#80 AS agg2#88, avg(agg3#31)#81 AS agg3#89, avg(agg4#32)#82 AS agg4#90, avg(agg5#33)#83 AS agg5#91, avg(agg6#34)#84 AS agg6#92, avg(agg7#35)#85 AS agg7#93] + +(66) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#121)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#94)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(70) CometFilter +(67) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(71) ColumnarToRow [codegen id : 23] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] - -(72) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(73) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None +(69) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Right output [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight -(74) Project [codegen id : 23] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +(70) CometProject Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(75) ReusedExchange [Reuses operator id: 15] +(71) ReusedExchange [Reuses operator id: 12] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(76) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None +(72) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight -(77) Project [codegen id : 23] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +(73) CometProject Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(78) ReusedExchange [Reuses operator id: 21] +(74) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#20] -(79) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(75) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [cd_demo_sk#20] +Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight -(80) Project [codegen id : 23] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +(76) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(81) Scan parquet spark_catalog.default.customer_address +(77) Scan parquet spark_catalog.default.customer_address Output [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(82) CometFilter +(78) CometFilter Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(83) CometProject +(79) CometProject Input [3]: [ca_address_sk#21, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_country#24], [ca_address_sk#21, ca_country#24] -(84) ColumnarToRow [codegen id : 20] +(80) CometBroadcastExchange Input [2]: [ca_address_sk#21, ca_country#24] +Arguments: [ca_address_sk#21, ca_country#24] -(85) BroadcastExchange -Input [2]: [ca_address_sk#21, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(86) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None +(81) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +Right output [2]: [ca_address_sk#21, ca_country#24] +Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight -(87) Project [codegen id : 23] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_country#24] +(82) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_country#24] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_country#24] -(88) ReusedExchange [Reuses operator id: 158] +(83) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#25] -(89) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(84) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_country#24] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#9], [d_date_sk#25], Inner, BuildRight -(90) Project [codegen id : 23] -Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24] +(85) CometProject Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_country#24, d_date_sk#25] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24] + +(86) ReusedExchange [Reuses operator id: 33] +Output [2]: [i_item_sk#27, i_item_id#28] + +(87) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24] +Right output [2]: [i_item_sk#27, i_item_id#28] +Arguments: [cs_item_sk#3], [i_item_sk#27], Inner, BuildRight + +(88) CometProject +Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24, i_item_sk#27, i_item_id#28] +Arguments: [i_item_id#28, ca_country#24, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35], [i_item_id#28, ca_country#24, cast(cs_quantity#4 as decimal(12,2)) AS agg1#29, cast(cs_list_price#5 as decimal(12,2)) AS agg2#30, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#31, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#32, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#33, cast(c_birth_year#19 as decimal(12,2)) AS agg6#34, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#35] + +(89) CometHashAggregate +Input [9]: [i_item_id#28, ca_country#24, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35] +Keys [2]: [i_item_id#28, ca_country#24] +Functions [7]: [partial_avg(agg1#29), partial_avg(agg2#30), partial_avg(agg3#31), partial_avg(agg4#32), partial_avg(agg5#33), partial_avg(agg6#34), partial_avg(agg7#35)] + +(90) ColumnarToRow [codegen id : 5] +Input [16]: [i_item_id#28, ca_country#24, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108] + +(91) Exchange +Input [16]: [i_item_id#28, ca_country#24, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108] +Arguments: hashpartitioning(i_item_id#28, ca_country#24, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(91) ReusedExchange [Reuses operator id: 36] -Output [2]: [i_item_sk#26, i_item_id#27] - -(92) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(93) Project [codegen id : 23] -Output [9]: [i_item_id#27, ca_country#24, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] -Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_country#24, i_item_sk#26, i_item_id#27] - -(94) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#27, ca_country#24, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] -Keys [2]: [i_item_id#27, ca_country#24] -Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] -Results [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] - -(95) Exchange -Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] -Arguments: hashpartitioning(i_item_id#27, ca_country#24, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(96) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#27, ca_country#24, sum#136, count#137, sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149] -Keys [2]: [i_item_id#27, ca_country#24] -Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] -Aggregate Attributes [7]: [avg(agg1#28)#150, avg(agg2#29)#151, avg(agg3#30)#152, avg(agg4#31)#153, avg(agg5#32)#154, avg(agg6#33)#155, avg(agg7#34)#156] -Results [11]: [i_item_id#27, ca_country#24, null AS ca_state#157, null AS county#158, avg(agg1#28)#150 AS agg1#159, avg(agg2#29)#151 AS agg2#160, avg(agg3#30)#152 AS agg3#161, avg(agg4#31)#153 AS agg4#162, avg(agg5#32)#154 AS agg5#163, avg(agg6#33)#155 AS agg6#164, avg(agg7#34)#156 AS agg7#165] - -(97) Scan parquet spark_catalog.default.catalog_sales +(92) HashAggregate [codegen id : 6] +Input [16]: [i_item_id#28, ca_country#24, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108] +Keys [2]: [i_item_id#28, ca_country#24] +Functions [7]: [avg(agg1#29), avg(agg2#30), avg(agg3#31), avg(agg4#32), avg(agg5#33), avg(agg6#34), avg(agg7#35)] +Aggregate Attributes [7]: [avg(agg1#29)#109, avg(agg2#30)#110, avg(agg3#31)#111, avg(agg4#32)#112, avg(agg5#33)#113, avg(agg6#34)#114, avg(agg7#35)#115] +Results [11]: [i_item_id#28, ca_country#24, null AS ca_state#116, null AS county#117, avg(agg1#29)#109 AS agg1#118, avg(agg2#30)#110 AS agg2#119, avg(agg3#31)#111 AS agg3#120, avg(agg4#32)#112 AS agg4#121, avg(agg5#33)#113 AS agg5#122, avg(agg6#34)#114 AS agg6#123, avg(agg7#35)#115 AS agg7#124] + +(93) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#166)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#125)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(98) CometFilter +(94) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(99) ColumnarToRow [codegen id : 31] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] - -(100) ReusedExchange [Reuses operator id: 8] +(95) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(101) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None +(96) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Right output [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight -(102) Project [codegen id : 31] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +(97) CometProject Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(103) ReusedExchange [Reuses operator id: 15] +(98) ReusedExchange [Reuses operator id: 12] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(104) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None +(99) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight -(105) Project [codegen id : 31] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +(100) CometProject Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(106) ReusedExchange [Reuses operator id: 21] +(101) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#20] -(107) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(102) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [cd_demo_sk#20] +Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight -(108) Project [codegen id : 31] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +(103) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(109) Scan parquet spark_catalog.default.customer_address +(104) Scan parquet spark_catalog.default.customer_address Output [2]: [ca_address_sk#21, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] ReadSchema: struct -(110) CometFilter +(105) CometFilter Input [2]: [ca_address_sk#21, ca_state#23] Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) -(111) CometProject +(106) CometProject Input [2]: [ca_address_sk#21, ca_state#23] Arguments: [ca_address_sk#21], [ca_address_sk#21] -(112) ColumnarToRow [codegen id : 28] -Input [1]: [ca_address_sk#21] - -(113) BroadcastExchange +(107) CometBroadcastExchange Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +Arguments: [ca_address_sk#21] -(114) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None +(108) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [ca_address_sk#21] +Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight -(115) Project [codegen id : 31] -Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] +(109) CometProject Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] -(116) ReusedExchange [Reuses operator id: 158] +(110) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#25] -(117) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(111) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#9], [d_date_sk#25], Inner, BuildRight -(118) Project [codegen id : 31] -Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] +(112) CometProject Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, d_date_sk#25] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] + +(113) ReusedExchange [Reuses operator id: 33] +Output [2]: [i_item_sk#27, i_item_id#28] + +(114) CometBroadcastHashJoin +Left output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] +Right output [2]: [i_item_sk#27, i_item_id#28] +Arguments: [cs_item_sk#3], [i_item_sk#27], Inner, BuildRight + +(115) CometProject +Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_sk#27, i_item_id#28] +Arguments: [i_item_id#28, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35], [i_item_id#28, cast(cs_quantity#4 as decimal(12,2)) AS agg1#29, cast(cs_list_price#5 as decimal(12,2)) AS agg2#30, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#31, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#32, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#33, cast(c_birth_year#19 as decimal(12,2)) AS agg6#34, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#35] -(119) ReusedExchange [Reuses operator id: 36] -Output [2]: [i_item_sk#26, i_item_id#27] - -(120) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(121) Project [codegen id : 31] -Output [8]: [i_item_id#27, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] -Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_sk#26, i_item_id#27] - -(122) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#27, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] -Keys [1]: [i_item_id#27] -Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#167, count#168, sum#169, count#170, sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180] -Results [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] - -(123) Exchange -Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(124) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#27, sum#181, count#182, sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194] -Keys [1]: [i_item_id#27] -Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] -Aggregate Attributes [7]: [avg(agg1#28)#195, avg(agg2#29)#196, avg(agg3#30)#197, avg(agg4#31)#198, avg(agg5#32)#199, avg(agg6#33)#200, avg(agg7#34)#201] -Results [11]: [i_item_id#27, null AS ca_country#202, null AS ca_state#203, null AS county#204, avg(agg1#28)#195 AS agg1#205, avg(agg2#29)#196 AS agg2#206, avg(agg3#30)#197 AS agg3#207, avg(agg4#31)#198 AS agg4#208, avg(agg5#32)#199 AS agg5#209, avg(agg6#33)#200 AS agg6#210, avg(agg7#34)#201 AS agg7#211] - -(125) Scan parquet spark_catalog.default.catalog_sales +(116) CometHashAggregate +Input [8]: [i_item_id#28, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35] +Keys [1]: [i_item_id#28] +Functions [7]: [partial_avg(agg1#29), partial_avg(agg2#30), partial_avg(agg3#31), partial_avg(agg4#32), partial_avg(agg5#33), partial_avg(agg6#34), partial_avg(agg7#35)] + +(117) ColumnarToRow [codegen id : 7] +Input [15]: [i_item_id#28, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] + +(118) Exchange +Input [15]: [i_item_id#28, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] +Arguments: hashpartitioning(i_item_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(119) HashAggregate [codegen id : 8] +Input [15]: [i_item_id#28, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139] +Keys [1]: [i_item_id#28] +Functions [7]: [avg(agg1#29), avg(agg2#30), avg(agg3#31), avg(agg4#32), avg(agg5#33), avg(agg6#34), avg(agg7#35)] +Aggregate Attributes [7]: [avg(agg1#29)#140, avg(agg2#30)#141, avg(agg3#31)#142, avg(agg4#32)#143, avg(agg5#33)#144, avg(agg6#34)#145, avg(agg7#35)#146] +Results [11]: [i_item_id#28, null AS ca_country#147, null AS ca_state#148, null AS county#149, avg(agg1#29)#140 AS agg1#150, avg(agg2#30)#141 AS agg2#151, avg(agg3#31)#142 AS agg3#152, avg(agg4#32)#143 AS agg4#153, avg(agg5#33)#144 AS agg5#154, avg(agg6#34)#145 AS agg6#155, avg(agg7#35)#146 AS agg7#156] + +(120) Scan parquet spark_catalog.default.catalog_sales Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#212)] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#157)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(126) CometFilter +(121) CometFilter Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) -(127) ColumnarToRow [codegen id : 39] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] - -(128) ReusedExchange [Reuses operator id: 8] +(122) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#11, cd_dep_count#14] -(129) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None +(123) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Right output [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight -(130) Project [codegen id : 39] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +(124) CometProject Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] +Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -(131) ReusedExchange [Reuses operator id: 15] +(125) ReusedExchange [Reuses operator id: 12] Output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(132) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None +(126) CometBroadcastHashJoin +Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight -(133) Project [codegen id : 39] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +(127) CometProject Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(134) ReusedExchange [Reuses operator id: 21] +(128) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#20] -(135) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None +(129) CometBroadcastHashJoin +Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [cd_demo_sk#20] +Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight -(136) Project [codegen id : 39] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +(130) CometProject Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -(137) ReusedExchange [Reuses operator id: 113] +(131) ReusedExchange [Reuses operator id: 107] Output [1]: [ca_address_sk#21] -(138) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None +(132) CometBroadcastHashJoin +Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +Right output [1]: [ca_address_sk#21] +Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight -(139) Project [codegen id : 39] -Output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] +(133) CometProject Input [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] -(140) ReusedExchange [Reuses operator id: 158] +(134) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#25] -(141) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#25] -Join type: Inner -Join condition: None +(135) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19] +Right output [1]: [d_date_sk#25] +Arguments: [cs_sold_date_sk#9], [d_date_sk#25], Inner, BuildRight -(142) Project [codegen id : 39] -Output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] +(136) CometProject Input [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, d_date_sk#25] +Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] -(143) Scan parquet spark_catalog.default.item -Output [1]: [i_item_sk#26] +(137) Scan parquet spark_catalog.default.item +Output [1]: [i_item_sk#27] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(144) CometFilter -Input [1]: [i_item_sk#26] -Condition : isnotnull(i_item_sk#26) +(138) CometFilter +Input [1]: [i_item_sk#27] +Condition : isnotnull(i_item_sk#27) -(145) ColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#26] +(139) CometBroadcastExchange +Input [1]: [i_item_sk#27] +Arguments: [i_item_sk#27] -(146) BroadcastExchange -Input [1]: [i_item_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +(140) CometBroadcastHashJoin +Left output [8]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19] +Right output [1]: [i_item_sk#27] +Arguments: [cs_item_sk#3], [i_item_sk#27], Inner, BuildRight -(147) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None +(141) CometProject +Input [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_sk#27] +Arguments: [agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35], [cast(cs_quantity#4 as decimal(12,2)) AS agg1#29, cast(cs_list_price#5 as decimal(12,2)) AS agg2#30, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#31, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#32, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#33, cast(c_birth_year#19 as decimal(12,2)) AS agg6#34, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#35] -(148) Project [codegen id : 39] -Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] -Input [9]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_sk#26] - -(149) HashAggregate [codegen id : 39] -Input [7]: [agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] +(142) CometHashAggregate +Input [7]: [agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35] Keys: [] -Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] -Aggregate Attributes [14]: [sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224, sum#225, count#226] -Results [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] +Functions [7]: [partial_avg(agg1#29), partial_avg(agg2#30), partial_avg(agg3#31), partial_avg(agg4#32), partial_avg(agg5#33), partial_avg(agg6#34), partial_avg(agg7#35)] + +(143) ColumnarToRow [codegen id : 9] +Input [14]: [sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165, sum#166, count#167, sum#168, count#169, sum#170, count#171] -(150) Exchange -Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] +(144) Exchange +Input [14]: [sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165, sum#166, count#167, sum#168, count#169, sum#170, count#171] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(151) HashAggregate [codegen id : 40] -Input [14]: [sum#227, count#228, sum#229, count#230, sum#231, count#232, sum#233, count#234, sum#235, count#236, sum#237, count#238, sum#239, count#240] +(145) HashAggregate [codegen id : 10] +Input [14]: [sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165, sum#166, count#167, sum#168, count#169, sum#170, count#171] Keys: [] -Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] -Aggregate Attributes [7]: [avg(agg1#28)#241, avg(agg2#29)#242, avg(agg3#30)#243, avg(agg4#31)#244, avg(agg5#32)#245, avg(agg6#33)#246, avg(agg7#34)#247] -Results [11]: [null AS i_item_id#248, null AS ca_country#249, null AS ca_state#250, null AS county#251, avg(agg1#28)#241 AS agg1#252, avg(agg2#29)#242 AS agg2#253, avg(agg3#30)#243 AS agg3#254, avg(agg4#31)#244 AS agg4#255, avg(agg5#32)#245 AS agg5#256, avg(agg6#33)#246 AS agg6#257, avg(agg7#34)#247 AS agg7#258] +Functions [7]: [avg(agg1#29), avg(agg2#30), avg(agg3#31), avg(agg4#32), avg(agg5#33), avg(agg6#34), avg(agg7#35)] +Aggregate Attributes [7]: [avg(agg1#29)#172, avg(agg2#30)#173, avg(agg3#31)#174, avg(agg4#32)#175, avg(agg5#33)#176, avg(agg6#34)#177, avg(agg7#35)#178] +Results [11]: [null AS i_item_id#179, null AS ca_country#180, null AS ca_state#181, null AS county#182, avg(agg1#29)#172 AS agg1#183, avg(agg2#30)#173 AS agg2#184, avg(agg3#31)#174 AS agg3#185, avg(agg4#32)#175 AS agg4#186, avg(agg5#33)#176 AS agg5#187, avg(agg6#34)#177 AS agg6#188, avg(agg7#35)#178 AS agg7#189] -(152) Union +(146) Union -(153) TakeOrderedAndProject -Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] -Arguments: 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#27 ASC NULLS FIRST], [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] +(147) TakeOrderedAndProject +Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#57, agg2#58, agg3#59, agg4#60, agg5#61, agg6#62, agg7#63] +Arguments: 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#57, agg2#58, agg3#59, agg4#60, agg5#61, agg6#62, agg7#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (158) -+- * ColumnarToRow (157) - +- CometProject (156) - +- CometFilter (155) - +- CometScan parquet spark_catalog.default.date_dim (154) +BroadcastExchange (152) ++- * ColumnarToRow (151) + +- CometProject (150) + +- CometFilter (149) + +- CometScan parquet spark_catalog.default.date_dim (148) -(154) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#259] +(148) Scan parquet spark_catalog.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,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(155) CometFilter -Input [2]: [d_date_sk#25, d_year#259] -Condition : ((isnotnull(d_year#259) AND (d_year#259 = 2001)) AND isnotnull(d_date_sk#25)) +(149) CometFilter +Input [2]: [d_date_sk#25, d_year#26] +Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#25)) -(156) CometProject -Input [2]: [d_date_sk#25, d_year#259] +(150) CometProject +Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(157) ColumnarToRow [codegen id : 1] +(151) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(158) BroadcastExchange +(152) BroadcastExchange Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 42 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 97 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 120 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index f02809572c..d4deedf0ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt @@ -1,25 +1,25 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] Union - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [i_item_id,ca_country,ca_state,ca_county,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),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,ca_country,ca_state,ca_county] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -30,204 +30,150 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_gender,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (16) + CometBroadcastExchange #3 + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometBroadcastExchange #4 + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometBroadcastExchange #5 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange #6 + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometBroadcastExchange #7 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #8 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (4) 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 - Exchange [i_item_id,ca_country,ca_state] #8 - WholeStageCodegen (15) - HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + Exchange [i_item_id,ca_country,ca_state] #9 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (24) + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange #10 + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + WholeStageCodegen (6) 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 - Exchange [i_item_id,ca_country] #10 - WholeStageCodegen (23) - HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + Exchange [i_item_id,ca_country] #11 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (20) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (32) + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange #12 + CometProject [ca_address_sk,ca_country] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + WholeStageCodegen (8) 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 - Exchange [i_item_id] #12 - WholeStageCodegen (31) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + Exchange [i_item_id] #13 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (28) - ColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_state,ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (40) + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + CometBroadcastExchange #14 + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + ReusedExchange [d_date_sk] #7 + ReusedExchange [i_item_sk,i_item_id] #8 + WholeStageCodegen (10) HashAggregate [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),i_item_id,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 - Exchange #14 - WholeStageCodegen (39) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + Exchange #15 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometProject [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometBroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #13 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (38) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + ReusedExchange [cd_demo_sk] #5 + ReusedExchange [ca_address_sk] #14 + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange #16 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index 1304af2e10..bfb24de435 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -1,24 +1,26 @@ == Physical Plan == -TakeOrderedAndProject (20) -+- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +TakeOrderedAndProject (22) ++- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * ColumnarToRow (15) + +- CometHashAggregate (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.catalog_sales @@ -33,118 +35,127 @@ ReadSchema: struct Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [cs_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) CometHashAggregate Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(15) ColumnarToRow [codegen id : 1] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16] -(16) Exchange +(18) Exchange Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] +(19) Sort [codegen id : 3] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window +(20) Window Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(19) Project [codegen id : 6] +(21) Project [codegen id : 4] Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] -(20) TakeOrderedAndProject +(22) TakeOrderedAndProject Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (25) -+- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.date_dim (21) +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) -(21) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(23) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(22) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(23) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(25) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(24) ColumnarToRow [codegen id : 1] +(26) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(25) BroadcastExchange +(27) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index 2a2a392cd0..513e6f9791 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,12 +29,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #4 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt index fdebdc8a4e..2810779ed7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt @@ -1,26 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * Expand (18) - +- * Project (17) - +- * BroadcastNestedLoopJoin Inner BuildRight (16) - :- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (15) - +- * ColumnarToRow (14) - +- CometScan parquet spark_catalog.default.warehouse (13) +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * Expand (20) + +- * Project (19) + +- * BroadcastNestedLoopJoin Inner BuildRight (18) + :- * ColumnarToRow (14) + : +- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.item (9) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometScan parquet spark_catalog.default.warehouse (15) (1) Scan parquet spark_catalog.default.inventory @@ -35,127 +37,138 @@ ReadSchema: struct Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] Condition : isnotnull(inv_item_sk#1) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 27] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] -(6) Project [codegen id : 4] -Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] +(7) CometBroadcastHashJoin +Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] +Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] -(7) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +(9) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Condition : isnotnull(i_item_sk#6) +(10) CometFilter +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Condition : isnotnull(i_item_sk#7) -(9) ColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +(11) CometBroadcastExchange +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(10) BroadcastExchange -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] +Right output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(13) CometProject +Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11], [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(12) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +(14) ColumnarToRow [codegen id : 2] +Input [5]: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(13) Scan parquet spark_catalog.default.warehouse +(15) Scan parquet spark_catalog.default.warehouse Output: [] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(14) ColumnarToRow [codegen id : 3] +(16) ColumnarToRow [codegen id : 1] Input: [] -(15) BroadcastExchange +(17) BroadcastExchange Input: [] -Arguments: IdentityBroadcastMode, [plan_id=2] +Arguments: IdentityBroadcastMode, [plan_id=1] -(16) BroadcastNestedLoopJoin [codegen id : 4] +(18) BroadcastNestedLoopJoin [codegen id : 2] Join type: Inner Join condition: None -(17) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, i_category#9] -Input [5]: [inv_quantity_on_hand#2, i_brand#7, i_class#8, i_category#9, i_product_name#10] +(19) Project [codegen id : 2] +Output [5]: [inv_quantity_on_hand#2, i_product_name#11, i_brand#8, i_class#9, i_category#10] +Input [5]: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(18) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, i_category#9] -Arguments: [[inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, i_category#9, 0], [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, null, 1], [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, null, null, 3], [inv_quantity_on_hand#2, i_product_name#10, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] +(20) Expand [codegen id : 2] +Input [5]: [inv_quantity_on_hand#2, i_product_name#11, i_brand#8, i_class#9, i_category#10] +Arguments: [[inv_quantity_on_hand#2, i_product_name#11, i_brand#8, i_class#9, i_category#10, 0], [inv_quantity_on_hand#2, i_product_name#11, i_brand#8, i_class#9, null, 1], [inv_quantity_on_hand#2, i_product_name#11, i_brand#8, null, null, 3], [inv_quantity_on_hand#2, i_product_name#11, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16] -(19) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#2, i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] -Keys [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] +(21) HashAggregate [codegen id : 2] +Input [6]: [inv_quantity_on_hand#2, i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16] +Keys [5]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#16, count#17] -Results [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] +Aggregate Attributes [2]: [sum#17, count#18] +Results [7]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16, sum#19, count#20] -(20) Exchange -Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) Exchange +Input [7]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16, sum#19, count#20] +Arguments: hashpartitioning(i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] -Keys [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] +(23) HashAggregate [codegen id : 3] +Input [7]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16, sum#19, count#20] +Keys [5]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, spark_grouping_id#16] Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#20] -Results [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, avg(inv_quantity_on_hand#2)#20 AS qoh#21] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#21] +Results [5]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, avg(inv_quantity_on_hand#2)#21 AS qoh#22] -(22) TakeOrderedAndProject -Input [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] -Arguments: 100, [qoh#21 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#12 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_category#14 ASC NULLS FIRST], [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] +(24) TakeOrderedAndProject +Input [5]: [i_product_name#12, i_brand#13, i_class#14, i_category#15, qoh#22] +Arguments: 100, [qoh#22 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_brand#13 ASC NULLS FIRST, i_class#14 ASC NULLS FIRST, i_category#15 ASC NULLS FIRST], [i_product_name#12, i_brand#13, i_class#14, i_category#15, qoh#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (27) -+- * ColumnarToRow (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan parquet spark_catalog.default.date_dim (23) +BroadcastExchange (29) ++- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) -(23) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#22] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(24) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#22] -Condition : (((isnotnull(d_month_seq#22) AND (d_month_seq#22 >= 1200)) AND (d_month_seq#22 <= 1211)) AND isnotnull(d_date_sk#5)) +(26) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(25) CometProject -Input [2]: [d_date_sk#5, d_month_seq#22] +(27) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(26) ColumnarToRow [codegen id : 1] +(28) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 63a428d4e4..415b430f86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] BroadcastNestedLoopJoin - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometProject [inv_item_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometFilter [inv_item_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -24,18 +24,16 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) + BroadcastExchange #5 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index 3270618dc7..08bad9380d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -1,49 +1,50 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- Union (44) - :- * HashAggregate (23) - : +- * HashAggregate (22) - : +- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.inventory (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.warehouse (13) - :- * HashAggregate (28) - : +- Exchange (27) - : +- * HashAggregate (26) - : +- * HashAggregate (25) - : +- ReusedExchange (24) - :- * HashAggregate (33) - : +- Exchange (32) - : +- * HashAggregate (31) - : +- * HashAggregate (30) - : +- ReusedExchange (29) - :- * HashAggregate (38) - : +- Exchange (37) - : +- * HashAggregate (36) - : +- * HashAggregate (35) - : +- ReusedExchange (34) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * HashAggregate (40) - +- ReusedExchange (39) +TakeOrderedAndProject (46) ++- Union (45) + :- * HashAggregate (24) + : +- * HashAggregate (23) + : +- * HashAggregate (22) + : +- Exchange (21) + : +- * ColumnarToRow (20) + : +- CometHashAggregate (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.item (9) + : +- CometBroadcastExchange (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.warehouse (14) + :- * HashAggregate (29) + : +- Exchange (28) + : +- * HashAggregate (27) + : +- * HashAggregate (26) + : +- ReusedExchange (25) + :- * HashAggregate (34) + : +- Exchange (33) + : +- * HashAggregate (32) + : +- * HashAggregate (31) + : +- ReusedExchange (30) + :- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * HashAggregate (36) + : +- ReusedExchange (35) + +- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- ReusedExchange (40) (1) Scan parquet spark_catalog.default.inventory @@ -58,258 +59,263 @@ ReadSchema: struct + +(4) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) -(4) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#6] +(5) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +(8) CometProject Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] +Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -(7) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(9) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) +(10) CometFilter +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Condition : isnotnull(i_item_sk#8) -(9) ColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +(11) CometBroadcastExchange +Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(10) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] +Right output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(13) CometProject +Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(12) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] - -(13) Scan parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#12] +(14) Scan parquet spark_catalog.default.warehouse +Output [1]: [w_warehouse_sk#13] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(14) CometFilter -Input [1]: [w_warehouse_sk#12] -Condition : isnotnull(w_warehouse_sk#12) - -(15) ColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#12] +(15) CometFilter +Input [1]: [w_warehouse_sk#13] +Condition : isnotnull(w_warehouse_sk#13) -(16) BroadcastExchange -Input [1]: [w_warehouse_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastExchange +Input [1]: [w_warehouse_sk#13] +Arguments: [w_warehouse_sk#13] -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#12] -Join type: Inner -Join condition: None +(17) CometBroadcastHashJoin +Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Right output [1]: [w_warehouse_sk#13] +Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#13], Inner, BuildRight -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11, w_warehouse_sk#12] +(18) CometProject +Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12, w_warehouse_sk#13] +Arguments: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12], [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +(19) CometHashAggregate +Input [5]: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#13, count#14] -Results [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -(20) Exchange -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, i_category#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(20) ColumnarToRow [codegen id : 1] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] + +(21) Exchange +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] +Arguments: hashpartitioning(i_product_name#12, i_brand#9, i_class#10, i_category#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +(22) HashAggregate [codegen id : 2] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(inv_quantity_on_hand#3)#17 AS qoh#18] - -(22) HashAggregate [codegen id : 5] -Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#18] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] -Functions [1]: [partial_avg(qoh#18)] -Aggregate Attributes [2]: [sum#19, count#20] -Results [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] - -(23) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] -Functions [1]: [avg(qoh#18)] -Aggregate Attributes [1]: [avg(qoh#18)#23] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(qoh#18)#23 AS qoh#24] - -(24) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] - -(25) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#16] +Results [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, avg(inv_quantity_on_hand#3)#16 AS qoh#17] + +(23) HashAggregate [codegen id : 2] +Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#17] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] +Functions [1]: [partial_avg(qoh#17)] +Aggregate Attributes [2]: [sum#18, count#19] +Results [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#20, count#21] + +(24) HashAggregate [codegen id : 2] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#20, count#21] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] +Functions [1]: [avg(qoh#17)] +Aggregate Attributes [1]: [avg(qoh#17)#22] +Results [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, avg(qoh#17)#22 AS qoh#23] + +(25) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] + +(26) HashAggregate [codegen id : 4] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] -Results [4]: [i_product_name#11, i_brand#8, i_class#9, avg(inv_quantity_on_hand#3)#17 AS qoh#18] - -(26) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#11, i_brand#8, i_class#9, qoh#18] -Keys [3]: [i_product_name#11, i_brand#8, i_class#9] -Functions [1]: [partial_avg(qoh#18)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] - -(27) Exchange -Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] -Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(28) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#11, i_brand#8, i_class#9, sum#27, count#28] -Keys [3]: [i_product_name#11, i_brand#8, i_class#9] -Functions [1]: [avg(qoh#18)] -Aggregate Attributes [1]: [avg(qoh#18)#29] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, null AS i_category#30, avg(qoh#18)#29 AS qoh#31] - -(29) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] - -(30) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#16] +Results [4]: [i_product_name#12, i_brand#9, i_class#10, avg(inv_quantity_on_hand#3)#16 AS qoh#17] + +(27) HashAggregate [codegen id : 4] +Input [4]: [i_product_name#12, i_brand#9, i_class#10, qoh#17] +Keys [3]: [i_product_name#12, i_brand#9, i_class#10] +Functions [1]: [partial_avg(qoh#17)] +Aggregate Attributes [2]: [sum#24, count#25] +Results [5]: [i_product_name#12, i_brand#9, i_class#10, sum#26, count#27] + +(28) Exchange +Input [5]: [i_product_name#12, i_brand#9, i_class#10, sum#26, count#27] +Arguments: hashpartitioning(i_product_name#12, i_brand#9, i_class#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(29) HashAggregate [codegen id : 5] +Input [5]: [i_product_name#12, i_brand#9, i_class#10, sum#26, count#27] +Keys [3]: [i_product_name#12, i_brand#9, i_class#10] +Functions [1]: [avg(qoh#17)] +Aggregate Attributes [1]: [avg(qoh#17)#28] +Results [5]: [i_product_name#12, i_brand#9, i_class#10, null AS i_category#29, avg(qoh#17)#28 AS qoh#30] + +(30) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] + +(31) HashAggregate [codegen id : 7] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] -Results [3]: [i_product_name#11, i_brand#8, avg(inv_quantity_on_hand#3)#17 AS qoh#18] - -(31) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#11, i_brand#8, qoh#18] -Keys [2]: [i_product_name#11, i_brand#8] -Functions [1]: [partial_avg(qoh#18)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [4]: [i_product_name#11, i_brand#8, sum#34, count#35] - -(32) Exchange -Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] -Arguments: hashpartitioning(i_product_name#11, i_brand#8, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(33) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#11, i_brand#8, sum#34, count#35] -Keys [2]: [i_product_name#11, i_brand#8] -Functions [1]: [avg(qoh#18)] -Aggregate Attributes [1]: [avg(qoh#18)#36] -Results [5]: [i_product_name#11, i_brand#8, null AS i_class#37, null AS i_category#38, avg(qoh#18)#36 AS qoh#39] - -(34) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] - -(35) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#16] +Results [3]: [i_product_name#12, i_brand#9, avg(inv_quantity_on_hand#3)#16 AS qoh#17] + +(32) HashAggregate [codegen id : 7] +Input [3]: [i_product_name#12, i_brand#9, qoh#17] +Keys [2]: [i_product_name#12, i_brand#9] +Functions [1]: [partial_avg(qoh#17)] +Aggregate Attributes [2]: [sum#31, count#32] +Results [4]: [i_product_name#12, i_brand#9, sum#33, count#34] + +(33) Exchange +Input [4]: [i_product_name#12, i_brand#9, sum#33, count#34] +Arguments: hashpartitioning(i_product_name#12, i_brand#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(34) HashAggregate [codegen id : 8] +Input [4]: [i_product_name#12, i_brand#9, sum#33, count#34] +Keys [2]: [i_product_name#12, i_brand#9] +Functions [1]: [avg(qoh#17)] +Aggregate Attributes [1]: [avg(qoh#17)#35] +Results [5]: [i_product_name#12, i_brand#9, null AS i_class#36, null AS i_category#37, avg(qoh#17)#35 AS qoh#38] + +(35) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] + +(36) HashAggregate [codegen id : 10] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] -Results [2]: [i_product_name#11, avg(inv_quantity_on_hand#3)#17 AS qoh#18] - -(36) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#11, qoh#18] -Keys [1]: [i_product_name#11] -Functions [1]: [partial_avg(qoh#18)] -Aggregate Attributes [2]: [sum#40, count#41] -Results [3]: [i_product_name#11, sum#42, count#43] - -(37) Exchange -Input [3]: [i_product_name#11, sum#42, count#43] -Arguments: hashpartitioning(i_product_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(38) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#11, sum#42, count#43] -Keys [1]: [i_product_name#11] -Functions [1]: [avg(qoh#18)] -Aggregate Attributes [1]: [avg(qoh#18)#44] -Results [5]: [i_product_name#11, null AS i_brand#45, null AS i_class#46, null AS i_category#47, avg(qoh#18)#44 AS qoh#48] - -(39) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] - -(40) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] -Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#16] +Results [2]: [i_product_name#12, avg(inv_quantity_on_hand#3)#16 AS qoh#17] + +(37) HashAggregate [codegen id : 10] +Input [2]: [i_product_name#12, qoh#17] +Keys [1]: [i_product_name#12] +Functions [1]: [partial_avg(qoh#17)] +Aggregate Attributes [2]: [sum#39, count#40] +Results [3]: [i_product_name#12, sum#41, count#42] + +(38) Exchange +Input [3]: [i_product_name#12, sum#41, count#42] +Arguments: hashpartitioning(i_product_name#12, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(39) HashAggregate [codegen id : 11] +Input [3]: [i_product_name#12, sum#41, count#42] +Keys [1]: [i_product_name#12] +Functions [1]: [avg(qoh#17)] +Aggregate Attributes [1]: [avg(qoh#17)#43] +Results [5]: [i_product_name#12, null AS i_brand#44, null AS i_class#45, null AS i_category#46, avg(qoh#17)#43 AS qoh#47] + +(40) ReusedExchange [Reuses operator id: 21] +Output [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] + +(41) HashAggregate [codegen id : 13] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#14, count#15] +Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] -Results [1]: [avg(inv_quantity_on_hand#3)#17 AS qoh#18] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#16] +Results [1]: [avg(inv_quantity_on_hand#3)#16 AS qoh#17] -(41) HashAggregate [codegen id : 28] -Input [1]: [qoh#18] +(42) HashAggregate [codegen id : 13] +Input [1]: [qoh#17] Keys: [] -Functions [1]: [partial_avg(qoh#18)] -Aggregate Attributes [2]: [sum#49, count#50] -Results [2]: [sum#51, count#52] +Functions [1]: [partial_avg(qoh#17)] +Aggregate Attributes [2]: [sum#48, count#49] +Results [2]: [sum#50, count#51] -(42) Exchange -Input [2]: [sum#51, count#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(43) Exchange +Input [2]: [sum#50, count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(43) HashAggregate [codegen id : 29] -Input [2]: [sum#51, count#52] +(44) HashAggregate [codegen id : 14] +Input [2]: [sum#50, count#51] Keys: [] -Functions [1]: [avg(qoh#18)] -Aggregate Attributes [1]: [avg(qoh#18)#53] -Results [5]: [null AS i_product_name#54, null AS i_brand#55, null AS i_class#56, null AS i_category#57, avg(qoh#18)#53 AS qoh#58] +Functions [1]: [avg(qoh#17)] +Aggregate Attributes [1]: [avg(qoh#17)#52] +Results [5]: [null AS i_product_name#53, null AS i_brand#54, null AS i_class#55, null AS i_category#56, avg(qoh#17)#52 AS qoh#57] -(44) Union +(45) Union -(45) TakeOrderedAndProject -Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] -Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] +(46) TakeOrderedAndProject +Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#23] +Arguments: 100, [qoh#23 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_brand#9 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#23] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (50) -+- * ColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +BroadcastExchange (51) ++- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#59] +(47) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(47) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#59] -Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1212)) AND (d_month_seq#59 <= 1223)) AND isnotnull(d_date_sk#6)) +(48) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) -(48) CometProject -Input [2]: [d_date_sk#6, d_month_seq#59] +(49) CometProject +Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(49) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(50) BroadcastExchange +(51) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index a8d71b06ac..6ab6c4fd77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter Exchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] + CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [inv_item_sk,i_item_sk] + CometProject [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + CometBroadcastHashJoin [inv_date_sk,d_date_sk] CometFilter [inv_item_sk,inv_warehouse_sk] CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -26,54 +26,48 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] - WholeStageCodegen (11) + CometBroadcastExchange #3 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #4 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + WholeStageCodegen (5) HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] InputAdapter - Exchange [i_product_name,i_brand,i_class] #5 - WholeStageCodegen (10) + Exchange [i_product_name,i_brand,i_class] #6 + WholeStageCodegen (4) HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] InputAdapter - Exchange [i_product_name,i_brand] #6 - WholeStageCodegen (16) + Exchange [i_product_name,i_brand] #7 + WholeStageCodegen (7) HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (23) + WholeStageCodegen (11) HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] InputAdapter - Exchange [i_product_name] #7 - WholeStageCodegen (22) + Exchange [i_product_name] #8 + WholeStageCodegen (10) HashAggregate [i_product_name,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (29) + WholeStageCodegen (14) HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] InputAdapter - Exchange #8 - WholeStageCodegen (28) + Exchange #9 + WholeStageCodegen (13) HashAggregate [qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt index 03399fd5a7..4abfc188bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt @@ -1,77 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (73) -+- Union (72) +TakeOrderedAndProject (71) ++- Union (70) :- * HashAggregate (28) : +- Exchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * ColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (23) - : +- * ColumnarToRow (22) + : +- * ColumnarToRow (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (14) + : : : +- CometBroadcastHashJoin (13) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (3) + : : : +- CometBroadcastExchange (12) + : : : +- CometProject (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (17) + : : +- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.store (15) + : +- CometBroadcastExchange (22) : +- CometFilter (21) : +- CometScan parquet spark_catalog.default.item (20) - :- * HashAggregate (50) - : +- Exchange (49) - : +- * HashAggregate (48) - : +- * Project (47) - : +- * BroadcastHashJoin Inner BuildRight (46) - : :- * Project (44) - : : +- * BroadcastHashJoin Inner BuildRight (43) - : : :- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * ColumnarToRow (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) - : : : : +- ReusedExchange (32) - : : : +- ReusedExchange (35) - : : +- BroadcastExchange (42) - : : +- * ColumnarToRow (41) - : : +- CometProject (40) - : : +- CometFilter (39) - : : +- CometScan parquet spark_catalog.default.store (38) - : +- ReusedExchange (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Project (56) - : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : :- * ColumnarToRow (53) - : : : : +- CometFilter (52) - : : : : +- CometScan parquet spark_catalog.default.store_sales (51) - : : : +- ReusedExchange (54) - : : +- ReusedExchange (57) - : +- ReusedExchange (60) - +- BroadcastExchange (66) - +- * ColumnarToRow (65) - +- CometFilter (64) - +- CometScan parquet spark_catalog.default.item (63) + :- * HashAggregate (49) + : +- Exchange (48) + : +- * ColumnarToRow (47) + : +- CometHashAggregate (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometProject (42) + : : +- CometBroadcastHashJoin (41) + : : :- CometProject (36) + : : : +- CometBroadcastHashJoin (35) + : : : :- CometProject (33) + : : : : +- CometBroadcastHashJoin (32) + : : : : :- CometFilter (30) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) + : : : : +- ReusedExchange (31) + : : : +- ReusedExchange (34) + : : +- CometBroadcastExchange (40) + : : +- CometProject (39) + : : +- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.store (37) + : +- ReusedExchange (43) + +- * HashAggregate (69) + +- Exchange (68) + +- * ColumnarToRow (67) + +- CometHashAggregate (66) + +- CometProject (65) + +- CometBroadcastHashJoin (64) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (57) + : : +- CometBroadcastHashJoin (56) + : : :- CometProject (54) + : : : +- CometBroadcastHashJoin (53) + : : : :- CometFilter (51) + : : : : +- CometScan parquet spark_catalog.default.store_sales (50) + : : : +- ReusedExchange (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (58) + +- CometBroadcastExchange (63) + +- CometFilter (62) + +- CometScan parquet spark_catalog.default.item (61) (1) Scan parquet spark_catalog.default.store_sales @@ -86,133 +84,134 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = F)) AND (cd_marital_status#12 = W)) AND (cd_education_status#13 = Primary )) AND isnotnull(cd_demo_sk#10)) -(6) CometProject +(5) CometProject Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) ColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [1]: [cd_demo_sk#10] +Arguments: [cd_demo_sk#10] -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(7) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [cd_demo_sk#10] +Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +(8) CometProject Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(11) ReusedExchange [Reuses operator id: 78] -Output [1]: [d_date_sk#14] +(9) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +(11) CometProject +Input [2]: [d_date_sk#14, d_year#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(12) CometBroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: [d_date_sk#14] + +(13) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight + +(14) CometProject Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(14) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] +(15) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#16, s_state#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((isnotnull(s_state#16) AND (s_state#16 = TN)) AND isnotnull(s_store_sk#15)) - -(16) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#16] +(16) CometFilter +Input [2]: [s_store_sk#16, s_state#17] +Condition : ((isnotnull(s_state#17) AND (s_state#17 = TN)) AND isnotnull(s_store_sk#16)) -(17) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [2]: [s_store_sk#16, s_state#17] +Arguments: [s_store_sk#16, s_state#17] -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [2]: [s_store_sk#16, s_state#17] +Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight -(19) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#16] +(19) CometProject +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#17] +Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] (20) Scan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_item_id#18] +Output [2]: [i_item_sk#18, i_item_id#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (21) CometFilter -Input [2]: [i_item_sk#17, i_item_id#18] -Condition : isnotnull(i_item_sk#17) +Input [2]: [i_item_sk#18, i_item_id#19] +Condition : isnotnull(i_item_sk#18) -(22) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#17, i_item_id#18] +(22) CometBroadcastExchange +Input [2]: [i_item_sk#18, i_item_id#19] +Arguments: [i_item_sk#18, i_item_id#19] -(23) BroadcastExchange -Input [2]: [i_item_sk#17, i_item_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] +Right output [2]: [i_item_sk#18, i_item_id#19] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(24) CometProject +Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#19] +Arguments: [i_item_id#19, s_state#17, agg1#20, agg2#21, agg3#22, agg4#23], [i_item_id#19, s_state#17, ss_quantity#4 AS agg1#20, ss_list_price#5 AS agg2#21, ss_coupon_amt#7 AS agg3#22, ss_sales_price#6 AS agg4#23] -(25) Project [codegen id : 5] -Output [6]: [i_item_id#18, s_state#16, ss_quantity#4 AS agg1#19, ss_list_price#5 AS agg2#20, ss_coupon_amt#7 AS agg3#21, ss_sales_price#6 AS agg4#22] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16, i_item_sk#17, i_item_id#18] +(25) CometHashAggregate +Input [6]: [i_item_id#19, s_state#17, agg1#20, agg2#21, agg3#22, agg4#23] +Keys [2]: [i_item_id#19, s_state#17] +Functions [4]: [partial_avg(agg1#20), partial_avg(UnscaledValue(agg2#21)), partial_avg(UnscaledValue(agg3#22)), partial_avg(UnscaledValue(agg4#23))] -(26) HashAggregate [codegen id : 5] -Input [6]: [i_item_id#18, s_state#16, agg1#19, agg2#20, agg3#21, agg4#22] -Keys [2]: [i_item_id#18, s_state#16] -Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), partial_avg(UnscaledValue(agg3#21)), partial_avg(UnscaledValue(agg4#22))] -Aggregate Attributes [8]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30] -Results [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] +(26) ColumnarToRow [codegen id : 1] +Input [10]: [i_item_id#19, s_state#17, sum#24, count#25, sum#26, count#27, sum#28, count#29, sum#30, count#31] (27) Exchange -Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] -Arguments: hashpartitioning(i_item_id#18, s_state#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [10]: [i_item_id#19, s_state#17, sum#24, count#25, sum#26, count#27, sum#28, count#29, sum#30, count#31] +Arguments: hashpartitioning(i_item_id#19, s_state#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(28) HashAggregate [codegen id : 6] -Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] -Keys [2]: [i_item_id#18, s_state#16] -Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] -Aggregate Attributes [4]: [avg(agg1#19)#39, avg(UnscaledValue(agg2#20))#40, avg(UnscaledValue(agg3#21))#41, avg(UnscaledValue(agg4#22))#42] -Results [7]: [i_item_id#18, s_state#16, 0 AS g_state#43, avg(agg1#19)#39 AS agg1#44, cast((avg(UnscaledValue(agg2#20))#40 / 100.0) as decimal(11,6)) AS agg2#45, cast((avg(UnscaledValue(agg3#21))#41 / 100.0) as decimal(11,6)) AS agg3#46, cast((avg(UnscaledValue(agg4#22))#42 / 100.0) as decimal(11,6)) AS agg4#47] +(28) HashAggregate [codegen id : 2] +Input [10]: [i_item_id#19, s_state#17, sum#24, count#25, sum#26, count#27, sum#28, count#29, sum#30, count#31] +Keys [2]: [i_item_id#19, s_state#17] +Functions [4]: [avg(agg1#20), avg(UnscaledValue(agg2#21)), avg(UnscaledValue(agg3#22)), avg(UnscaledValue(agg4#23))] +Aggregate Attributes [4]: [avg(agg1#20)#32, avg(UnscaledValue(agg2#21))#33, avg(UnscaledValue(agg3#22))#34, avg(UnscaledValue(agg4#23))#35] +Results [7]: [i_item_id#19, s_state#17, 0 AS g_state#36, avg(agg1#20)#32 AS agg1#37, cast((avg(UnscaledValue(agg2#21))#33 / 100.0) as decimal(11,6)) AS agg2#38, cast((avg(UnscaledValue(agg3#22))#34 / 100.0) as decimal(11,6)) AS agg3#39, cast((avg(UnscaledValue(agg4#23))#35 / 100.0) as decimal(11,6)) AS agg4#40] (29) Scan parquet spark_catalog.default.store_sales Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#48)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#41)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct @@ -220,238 +219,220 @@ ReadSchema: struct -(39) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((isnotnull(s_state#16) AND (s_state#16 = TN)) AND isnotnull(s_store_sk#15)) - -(40) CometProject -Input [2]: [s_store_sk#15, s_state#16] -Arguments: [s_store_sk#15], [s_store_sk#15] - -(41) ColumnarToRow [codegen id : 9] -Input [1]: [s_store_sk#15] - -(42) BroadcastExchange -Input [1]: [s_store_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 11] -Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15] - -(45) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#17, i_item_id#18] - -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 11] -Output [5]: [i_item_id#18, ss_quantity#4 AS agg1#19, ss_list_price#5 AS agg2#20, ss_coupon_amt#7 AS agg3#21, ss_sales_price#6 AS agg4#22] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#17, i_item_id#18] - -(48) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#18, agg1#19, agg2#20, agg3#21, agg4#22] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), partial_avg(UnscaledValue(agg3#21)), partial_avg(UnscaledValue(agg4#22))] -Aggregate Attributes [8]: [sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56] -Results [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] - -(49) Exchange -Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(50) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#18, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] -Aggregate Attributes [4]: [avg(agg1#19)#65, avg(UnscaledValue(agg2#20))#66, avg(UnscaledValue(agg3#21))#67, avg(UnscaledValue(agg4#22))#68] -Results [7]: [i_item_id#18, null AS s_state#69, 1 AS g_state#70, avg(agg1#19)#65 AS agg1#71, cast((avg(UnscaledValue(agg2#20))#66 / 100.0) as decimal(11,6)) AS agg2#72, cast((avg(UnscaledValue(agg3#21))#67 / 100.0) as decimal(11,6)) AS agg3#73, cast((avg(UnscaledValue(agg4#22))#68 / 100.0) as decimal(11,6)) AS agg4#74] - -(51) Scan parquet spark_catalog.default.store_sales +(38) CometFilter +Input [2]: [s_store_sk#16, s_state#17] +Condition : ((isnotnull(s_state#17) AND (s_state#17 = TN)) AND isnotnull(s_store_sk#16)) + +(39) CometProject +Input [2]: [s_store_sk#16, s_state#17] +Arguments: [s_store_sk#16], [s_store_sk#16] + +(40) CometBroadcastExchange +Input [1]: [s_store_sk#16] +Arguments: [s_store_sk#16] + +(41) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [1]: [s_store_sk#16] +Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight + +(42) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16] +Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] + +(43) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#18, i_item_id#19] + +(44) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [2]: [i_item_sk#18, i_item_id#19] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight + +(45) CometProject +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#18, i_item_id#19] +Arguments: [i_item_id#19, agg1#20, agg2#21, agg3#22, agg4#23], [i_item_id#19, ss_quantity#4 AS agg1#20, ss_list_price#5 AS agg2#21, ss_coupon_amt#7 AS agg3#22, ss_sales_price#6 AS agg4#23] + +(46) CometHashAggregate +Input [5]: [i_item_id#19, agg1#20, agg2#21, agg3#22, agg4#23] +Keys [1]: [i_item_id#19] +Functions [4]: [partial_avg(agg1#20), partial_avg(UnscaledValue(agg2#21)), partial_avg(UnscaledValue(agg3#22)), partial_avg(UnscaledValue(agg4#23))] + +(47) ColumnarToRow [codegen id : 3] +Input [9]: [i_item_id#19, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] + +(48) Exchange +Input [9]: [i_item_id#19, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] +Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(49) HashAggregate [codegen id : 4] +Input [9]: [i_item_id#19, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] +Keys [1]: [i_item_id#19] +Functions [4]: [avg(agg1#20), avg(UnscaledValue(agg2#21)), avg(UnscaledValue(agg3#22)), avg(UnscaledValue(agg4#23))] +Aggregate Attributes [4]: [avg(agg1#20)#50, avg(UnscaledValue(agg2#21))#51, avg(UnscaledValue(agg3#22))#52, avg(UnscaledValue(agg4#23))#53] +Results [7]: [i_item_id#19, null AS s_state#54, 1 AS g_state#55, avg(agg1#20)#50 AS agg1#56, cast((avg(UnscaledValue(agg2#21))#51 / 100.0) as decimal(11,6)) AS agg2#57, cast((avg(UnscaledValue(agg3#22))#52 / 100.0) as decimal(11,6)) AS agg3#58, cast((avg(UnscaledValue(agg4#23))#53 / 100.0) as decimal(11,6)) AS agg4#59] + +(50) Scan parquet spark_catalog.default.store_sales Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#60)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(52) CometFilter +(51) CometFilter Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) -(53) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(54) ReusedExchange [Reuses operator id: 8] +(52) ReusedExchange [Reuses operator id: 6] Output [1]: [cd_demo_sk#10] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None +(53) CometBroadcastHashJoin +Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [cd_demo_sk#10] +Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight -(56) Project [codegen id : 17] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +(54) CometProject Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(57) ReusedExchange [Reuses operator id: 78] +(55) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#14] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None +(56) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Right output [1]: [d_date_sk#14] +Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight -(59) Project [codegen id : 17] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +(57) CometProject Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] +Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(60) ReusedExchange [Reuses operator id: 42] -Output [1]: [s_store_sk#15] +(58) ReusedExchange [Reuses operator id: 40] +Output [1]: [s_store_sk#16] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None +(59) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [1]: [s_store_sk#16] +Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight -(62) Project [codegen id : 17] -Output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15] +(60) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16] +Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -(63) Scan parquet spark_catalog.default.item -Output [1]: [i_item_sk#17] +(61) Scan parquet spark_catalog.default.item +Output [1]: [i_item_sk#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(64) CometFilter -Input [1]: [i_item_sk#17] -Condition : isnotnull(i_item_sk#17) +(62) CometFilter +Input [1]: [i_item_sk#18] +Condition : isnotnull(i_item_sk#18) -(65) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#17] +(63) CometBroadcastExchange +Input [1]: [i_item_sk#18] +Arguments: [i_item_sk#18] -(66) BroadcastExchange -Input [1]: [i_item_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(64) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +Right output [1]: [i_item_sk#18] +Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None +(65) CometProject +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#18] +Arguments: [agg1#20, agg2#21, agg3#22, agg4#23], [ss_quantity#4 AS agg1#20, ss_list_price#5 AS agg2#21, ss_coupon_amt#7 AS agg3#22, ss_sales_price#6 AS agg4#23] -(68) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#19, ss_list_price#5 AS agg2#20, ss_coupon_amt#7 AS agg3#21, ss_sales_price#6 AS agg4#22] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#17] - -(69) HashAggregate [codegen id : 17] -Input [4]: [agg1#19, agg2#20, agg3#21, agg4#22] +(66) CometHashAggregate +Input [4]: [agg1#20, agg2#21, agg3#22, agg4#23] Keys: [] -Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), partial_avg(UnscaledValue(agg3#21)), partial_avg(UnscaledValue(agg4#22))] -Aggregate Attributes [8]: [sum#76, count#77, sum#78, count#79, sum#80, count#81, sum#82, count#83] -Results [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] +Functions [4]: [partial_avg(agg1#20), partial_avg(UnscaledValue(agg2#21)), partial_avg(UnscaledValue(agg3#22)), partial_avg(UnscaledValue(agg4#23))] + +(67) ColumnarToRow [codegen id : 5] +Input [8]: [sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -(70) Exchange -Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] +(68) Exchange +Input [8]: [sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(71) HashAggregate [codegen id : 18] -Input [8]: [sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] +(69) HashAggregate [codegen id : 6] +Input [8]: [sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] Keys: [] -Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] -Aggregate Attributes [4]: [avg(agg1#19)#92, avg(UnscaledValue(agg2#20))#93, avg(UnscaledValue(agg3#21))#94, avg(UnscaledValue(agg4#22))#95] -Results [7]: [null AS i_item_id#96, null AS s_state#97, 1 AS g_state#98, avg(agg1#19)#92 AS agg1#99, cast((avg(UnscaledValue(agg2#20))#93 / 100.0) as decimal(11,6)) AS agg2#100, cast((avg(UnscaledValue(agg3#21))#94 / 100.0) as decimal(11,6)) AS agg3#101, cast((avg(UnscaledValue(agg4#22))#95 / 100.0) as decimal(11,6)) AS agg4#102] +Functions [4]: [avg(agg1#20), avg(UnscaledValue(agg2#21)), avg(UnscaledValue(agg3#22)), avg(UnscaledValue(agg4#23))] +Aggregate Attributes [4]: [avg(agg1#20)#69, avg(UnscaledValue(agg2#21))#70, avg(UnscaledValue(agg3#22))#71, avg(UnscaledValue(agg4#23))#72] +Results [7]: [null AS i_item_id#73, null AS s_state#74, 1 AS g_state#75, avg(agg1#20)#69 AS agg1#76, cast((avg(UnscaledValue(agg2#21))#70 / 100.0) as decimal(11,6)) AS agg2#77, cast((avg(UnscaledValue(agg3#22))#71 / 100.0) as decimal(11,6)) AS agg3#78, cast((avg(UnscaledValue(agg4#23))#72 / 100.0) as decimal(11,6)) AS agg4#79] -(72) Union +(70) Union -(73) TakeOrderedAndProject -Input [7]: [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] -Arguments: 100, [i_item_id#18 ASC NULLS FIRST, s_state#16 ASC NULLS FIRST], [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] +(71) TakeOrderedAndProject +Input [7]: [i_item_id#19, s_state#17, g_state#36, agg1#37, agg2#38, agg3#39, agg4#40] +Arguments: 100, [i_item_id#19 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#19, s_state#17, g_state#36, agg1#37, agg2#38, agg3#39, agg4#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (78) -+- * ColumnarToRow (77) - +- CometProject (76) - +- CometFilter (75) - +- CometScan parquet spark_catalog.default.date_dim (74) +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(74) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#103] +(72) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(75) CometFilter -Input [2]: [d_date_sk#14, d_year#103] -Condition : ((isnotnull(d_year#103) AND (d_year#103 = 1998)) AND isnotnull(d_date_sk#14)) +(73) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) -(76) CometProject -Input [2]: [d_date_sk#14, d_year#103] +(74) CometProject +Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(77) ColumnarToRow [codegen id : 1] +(75) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(78) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt index 32f003798d..371254b085 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter Exchange [i_item_id,s_state] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -26,92 +26,68 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (12) + CometBroadcastExchange #3 + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #5 + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter - Exchange [i_item_id] #6 - WholeStageCodegen (11) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,agg1,agg2,agg3,agg4] + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #8 + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [i_item_sk,i_item_id] #6 + WholeStageCodegen (6) HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] InputAdapter - Exchange #8 - WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - ColumnarToRow - InputAdapter + Exchange #9 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [agg1,agg2,agg3,agg4] + CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [s_store_sk] #7 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk] + ReusedExchange [cd_demo_sk] #3 + ReusedExchange [d_date_sk] #4 + ReusedExchange [s_store_sk] #8 + CometBroadcastExchange #10 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt index 387fa47f9b..92e11aabe8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt @@ -1,36 +1,37 @@ == Physical Plan == -* Sort (32) -+- Exchange (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (24) - : +- * HashAggregate (23) - : +- Exchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * ColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (28) - +- * ColumnarToRow (27) - +- CometFilter (26) - +- CometScan parquet spark_catalog.default.customer (25) +* Sort (33) ++- Exchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Filter (25) + : +- * HashAggregate (24) + : +- Exchange (23) + : +- * ColumnarToRow (22) + : +- CometHashAggregate (21) + : +- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometProject (14) + : : +- CometBroadcastHashJoin (13) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (12) + : : +- CometProject (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan parquet spark_catalog.default.household_demographics (15) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.customer (26) (1) Scan parquet spark_catalog.default.store_sales @@ -45,174 +46,179 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#7] +(4) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] + +(7) CometBroadcastHashJoin +Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +(8) CometProject Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#10, s_county#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] +(10) CometFilter +Input [2]: [s_store_sk#10, s_county#11] +Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) -(10) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] +(11) CometProject +Input [2]: [s_store_sk#10, s_county#11] +Arguments: [s_store_sk#10], [s_store_sk#10] -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +(12) CometBroadcastExchange +Input [1]: [s_store_sk#10] +Arguments: [s_store_sk#10] -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None +(13) CometBroadcastHashJoin +Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] +Right output [1]: [s_store_sk#10] +Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] +(14) CometProject +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] +Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -(14) Scan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +(15) Scan parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) +(16) CometFilter +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Condition : ((((isnotnull(hd_vehicle_count#15) AND ((hd_buy_potential#13 = >10000 ) OR (hd_buy_potential#13 = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN ((cast(hd_dep_count#14 as double) / cast(hd_vehicle_count#15 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#12)) -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] +(17) CometProject +Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] +Arguments: [hd_demo_sk#12], [hd_demo_sk#12] -(17) ColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] +(18) CometBroadcastExchange +Input [1]: [hd_demo_sk#12] +Arguments: [hd_demo_sk#12] -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(19) CometBroadcastHashJoin +Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Right output [1]: [hd_demo_sk#12] +Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None +(20) CometProject +Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] +Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] +(21) CometHashAggregate Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(22) Exchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(22) ColumnarToRow [codegen id : 1] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] + +(23) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(23) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +(24) HashAggregate [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] +Aggregate Attributes [1]: [count(1)#17] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#17 AS cnt#18] -(24) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) +(25) Filter [codegen id : 3] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18] +Condition : ((cnt#18 >= 15) AND (cnt#18 <= 20)) -(25) Scan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(26) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) +(27) CometFilter +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Condition : isnotnull(c_customer_sk#19) -(27) ColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(28) ColumnarToRow [codegen id : 2] +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(28) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(29) BroadcastExchange +Input [5]: [c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(29) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] +Right keys [1]: [c_customer_sk#19] Join type: Inner Join condition: None -(30) Project [codegen id : 6] -Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +(31) Project [codegen id : 3] +Output [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#18, c_customer_sk#19, c_salutation#20, c_first_name#21, c_last_name#22, c_preferred_cust_flag#23] -(31) Exchange -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(32) Exchange +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: rangepartitioning(c_last_name#22 ASC NULLS FIRST, c_first_name#21 ASC NULLS FIRST, c_salutation#20 ASC NULLS FIRST, c_preferred_cust_flag#23 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 7] -Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST], true, 0 +(33) Sort [codegen id : 4] +Input [6]: [c_last_name#22, c_first_name#21, c_salutation#20, c_preferred_cust_flag#23, ss_ticket_number#4, cnt#18] +Arguments: [c_last_name#22 ASC NULLS FIRST, c_first_name#21 ASC NULLS FIRST, c_salutation#20 ASC NULLS FIRST, c_preferred_cust_flag#23 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * ColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (38) ++- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(33) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +(34) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_dom#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] -Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) +(35) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] +Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) -(35) CometProject -Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +(36) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(37) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(37) BroadcastExchange +(38) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index b473e48921..970562e5c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -1,24 +1,24 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] InputAdapter Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] BroadcastHashJoin [ss_customer_sk,c_customer_sk] Filter [cnt] HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] InputAdapter Exchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_ticket_number,ss_customer_sk] + CometProject [ss_customer_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -29,27 +29,21 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_dom,d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_county,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometBroadcastExchange #5 + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + CometBroadcastExchange #6 + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) + BroadcastExchange #7 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [c_customer_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index ce8753277a..7900000853 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * ColumnarToRow (29) - : +- CometFilter (28) - : +- CometScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (36) - +- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.customer_demographics (33) +TakeOrderedAndProject (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (28) + : : +- * Filter (27) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) + : : : :- * ColumnarToRow (12) + : : : : +- CometBroadcastHashJoin (11) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (10) + : : : : +- CometProject (9) + : : : : +- CometBroadcastHashJoin (8) + : : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- CometBroadcastExchange (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * ColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) Scan parquet spark_catalog.default.customer @@ -54,228 +56,237 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct -(6) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#9] +(5) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(6) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(7) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] + +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] +(9) CometProject Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ss_customer_sk#6], [ss_customer_sk#6] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#6] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Right output [1]: [ss_customer_sk#6] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight + +(12) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(13) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +(14) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#15] -(13) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#13] +(15) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] +Right output [1]: [d_date_sk#15] +Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(16) CometProject +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] +Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +(17) ColumnarToRow [codegen id : 1] +Input [1]: [ws_bill_customer_sk#12] -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(18) BroadcastExchange +Input [1]: [ws_bill_customer_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(17) BroadcastHashJoin [codegen id : 9] +(19) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] +Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) Join condition: None -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(20) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +(21) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#19] -(20) ReusedExchange [Reuses operator id: 47] -Output [1]: [d_date_sk#17] +(22) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Right output [1]: [d_date_sk#19] +Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#15] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None +(23) CometProject +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] +Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#14] -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] +(24) ColumnarToRow [codegen id : 2] +Input [1]: [cs_ship_customer_sk#16] -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(25) BroadcastExchange +Input [1]: [cs_ship_customer_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 9] +(26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#14] +Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(27) Filter [codegen id : 5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(28) Project [codegen id : 5] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -(27) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +(29) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) +(30) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(29) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] +(31) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#20, ca_state#21] -(30) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(32) BroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(31) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(32) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#19] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#19] +(34) Project [codegen id : 5] +Output [2]: [c_current_cdemo_sk#4, ca_state#21] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] -(33) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(35) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(34) CometFilter -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Condition : isnotnull(cd_demo_sk#20) +(36) CometFilter +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Condition : isnotnull(cd_demo_sk#22) -(35) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(37) ColumnarToRow [codegen id : 4] +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(36) BroadcastExchange -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(38) BroadcastExchange +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#20] +Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 9] -Output [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Input [8]: [c_current_cdemo_sk#4, ca_state#19, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(40) Project [codegen id : 5] +Output [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(39) HashAggregate [codegen id : 9] -Input [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#23), partial_max(cd_dep_count#23), partial_sum(cd_dep_count#23), partial_avg(cd_dep_employed_count#24), partial_max(cd_dep_employed_count#24), partial_sum(cd_dep_employed_count#24), partial_avg(cd_dep_college_count#25), partial_max(cd_dep_college_count#25), partial_sum(cd_dep_college_count#25)] -Aggregate Attributes [13]: [count#26, sum#27, count#28, max#29, sum#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38] -Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] +(41) HashAggregate [codegen id : 5] +Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] +Aggregate Attributes [13]: [count#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40] +Results [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] -(40) Exchange -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(42) Exchange +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] +Arguments: hashpartitioning(ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(41) HashAggregate [codegen id : 10] -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] -Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] -Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] +(43) HashAggregate [codegen id : 6] +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#41, sum#42, count#43, max#44, sum#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] +Aggregate Attributes [10]: [count(1)#54, avg(cd_dep_count#25)#55, max(cd_dep_count#25)#56, sum(cd_dep_count#25)#57, avg(cd_dep_employed_count#26)#58, max(cd_dep_employed_count#26)#59, sum(cd_dep_employed_count#26)#60, avg(cd_dep_college_count#27)#61, max(cd_dep_college_count#27)#62, sum(cd_dep_college_count#27)#63] +Results [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#54 AS cnt1#64, avg(cd_dep_count#25)#55 AS avg(cd_dep_count)#65, max(cd_dep_count#25)#56 AS max(cd_dep_count)#66, sum(cd_dep_count#25)#57 AS sum(cd_dep_count)#67, cd_dep_employed_count#26, count(1)#54 AS cnt2#68, avg(cd_dep_employed_count#26)#58 AS avg(cd_dep_employed_count)#69, max(cd_dep_employed_count#26)#59 AS max(cd_dep_employed_count)#70, sum(cd_dep_employed_count#26)#60 AS sum(cd_dep_employed_count)#71, cd_dep_college_count#27, count(1)#54 AS cnt3#72, avg(cd_dep_college_count#27)#61 AS avg(cd_dep_college_count)#73, max(cd_dep_college_count#27)#62 AS max(cd_dep_college_count)#74, sum(cd_dep_college_count#27)#63 AS sum(cd_dep_college_count)#75] -(42) TakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] -Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] +(44) TakeOrderedAndProject +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#64, avg(cd_dep_count)#65, max(cd_dep_count)#66, sum(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, avg(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, sum(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, avg(cd_dep_college_count)#73, max(cd_dep_college_count)#74, sum(cd_dep_college_count)#75] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#64, avg(cd_dep_count)#65, max(cd_dep_count)#66, sum(cd_dep_count)#67, cd_dep_employed_count#26, cnt2#68, avg(cd_dep_employed_count)#69, max(cd_dep_employed_count)#70, sum(cd_dep_employed_count)#71, cd_dep_college_count#27, cnt3#72, avg(cd_dep_college_count)#73, max(cd_dep_college_count)#74, sum(cd_dep_college_count)#75] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) -(43) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#74, d_qoy#75] +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] -Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) +(46) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] +Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) -(45) CometProject -Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +(47) CometProject +Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(46) ColumnarToRow [codegen id : 1] +(48) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(47) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index dc724ca919..a6b4add1fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (6) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,61 +13,57 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Filter [exists,exists] BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ws_bill_customer_sk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [cs_ship_customer_sk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) + BroadcastExchange #8 + WholeStageCodegen (4) ColumnarToRow InputAdapter CometFilter [cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 648b19933c..215d4240e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -1,44 +1,42 @@ == Physical Plan == -TakeOrderedAndProject (40) -+- * HashAggregate (39) - +- Exchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- CometScan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (28) - : +- * ColumnarToRow (27) - : +- CometFilter (26) - : +- CometScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (34) - +- * ColumnarToRow (33) - +- CometFilter (32) - +- CometScan parquet spark_catalog.default.customer_demographics (31) +TakeOrderedAndProject (38) ++- * HashAggregate (37) + +- Exchange (36) + +- * ColumnarToRow (35) + +- CometHashAggregate (34) + +- CometProject (33) + +- CometBroadcastHashJoin (32) + :- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometBroadcastHashJoin (11) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : +- CometBroadcastExchange (10) + : : : +- CometProject (9) + : : : +- CometBroadcastHashJoin (8) + : : : :- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : +- CometBroadcastExchange (21) + : : +- CometUnion (20) + : : :- CometProject (15) + : : : +- CometBroadcastHashJoin (14) + : : : :- CometScan parquet spark_catalog.default.web_sales (12) + : : : +- ReusedExchange (13) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (17) + : +- CometBroadcastExchange (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer_address (24) + +- CometBroadcastExchange (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.customer_demographics (29) (1) Scan parquet spark_catalog.default.customer @@ -52,216 +50,210 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct + +(5) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) + +(6) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Arguments: [d_date_sk#7], [d_date_sk#7] -(6) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#7] +(7) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(8) CometBroadcastHashJoin +Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] +(9) CometProject Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_customer_sk#4], [ss_customer_sk#4] -(9) BroadcastExchange +(10) CometBroadcastExchange Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] +Arguments: [ss_customer_sk#4] -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None +(11) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [ss_customer_sk#4] +Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +(12) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#11] +(13) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#13] -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customsk#12] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] +(15) CometProject +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] +Arguments: [customsk#14], [ws_bill_customer_sk#10 AS customsk#14] (16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +(17) ReusedExchange [Reuses operator id: 7] +Output [1]: [d_date_sk#18] -(18) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#16] +(18) CometBroadcastHashJoin +Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Right output [1]: [d_date_sk#18] +Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None +(19) CometProject +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] +Arguments: [customsk#19], [cs_ship_customer_sk#15 AS customsk#19] -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customsk#17] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] +(20) CometUnion +Child 0 Input [1]: [customsk#14] +Child 1 Input [1]: [customsk#19] -(21) Union +(21) CometBroadcastExchange +Input [1]: [customsk#14] +Arguments: [customsk#14] -(22) BroadcastExchange -Input [1]: [customsk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(22) CometBroadcastHashJoin +Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [1]: [customsk#14] +Arguments: [c_customer_sk#1], [customsk#14], LeftSemi, BuildRight -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#12] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +(23) CometProject Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] -(25) Scan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +(24) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct -(26) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) - -(27) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] +(25) CometFilter +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) -(28) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(26) CometBroadcastExchange +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: [ca_address_sk#20, ca_state#21] -(29) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None +(27) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +Right output [2]: [ca_address_sk#20, ca_state#21] +Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight -(30) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#19] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18, ca_state#19] +(28) CometProject +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20, ca_state#21] +Arguments: [c_current_cdemo_sk#2, ca_state#21], [c_current_cdemo_sk#2, ca_state#21] -(31) Scan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +(29) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(32) CometFilter -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Condition : isnotnull(cd_demo_sk#20) - -(33) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] - -(34) BroadcastExchange -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 9] -Output [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Input [8]: [c_current_cdemo_sk#2, ca_state#19, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] - -(37) HashAggregate [codegen id : 9] -Input [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#23), partial_max(cd_dep_count#23), partial_sum(cd_dep_count#23), partial_avg(cd_dep_employed_count#24), partial_max(cd_dep_employed_count#24), partial_sum(cd_dep_employed_count#24), partial_avg(cd_dep_college_count#25), partial_max(cd_dep_college_count#25), partial_sum(cd_dep_college_count#25)] -Aggregate Attributes [13]: [count#26, sum#27, count#28, max#29, sum#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38] -Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] - -(38) Exchange -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(39) HashAggregate [codegen id : 10] -Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] -Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] -Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] - -(40) TakeOrderedAndProject -Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] -Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] +(30) CometFilter +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Condition : isnotnull(cd_demo_sk#22) + +(31) CometBroadcastExchange +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] + +(32) CometBroadcastHashJoin +Left output [2]: [c_current_cdemo_sk#2, ca_state#21] +Right output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight + +(33) CometProject +Input [8]: [c_current_cdemo_sk#2, ca_state#21, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27], [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] + +(34) CometHashAggregate +Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] + +(35) ColumnarToRow [codegen id : 1] +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40] + +(36) Exchange +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40] +Arguments: hashpartitioning(ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(37) HashAggregate [codegen id : 2] +Input [19]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#28, sum#29, count#30, max#31, sum#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40] +Keys [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] +Aggregate Attributes [10]: [count(1)#41, avg(cd_dep_count#25)#42, max(cd_dep_count#25)#43, sum(cd_dep_count#25)#44, avg(cd_dep_employed_count#26)#45, max(cd_dep_employed_count#26)#46, sum(cd_dep_employed_count#26)#47, avg(cd_dep_college_count#27)#48, max(cd_dep_college_count#27)#49, sum(cd_dep_college_count#27)#50] +Results [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#41 AS cnt1#51, avg(cd_dep_count#25)#42 AS avg(cd_dep_count)#52, max(cd_dep_count#25)#43 AS max(cd_dep_count)#53, sum(cd_dep_count#25)#44 AS sum(cd_dep_count)#54, cd_dep_employed_count#26, count(1)#41 AS cnt2#55, avg(cd_dep_employed_count#26)#45 AS avg(cd_dep_employed_count)#56, max(cd_dep_employed_count#26)#46 AS max(cd_dep_employed_count)#57, sum(cd_dep_employed_count#26)#47 AS sum(cd_dep_employed_count)#58, cd_dep_college_count#27, count(1)#41 AS cnt3#59, avg(cd_dep_college_count#27)#48 AS avg(cd_dep_college_count)#60, max(cd_dep_college_count#27)#49 AS max(cd_dep_college_count)#61, sum(cd_dep_college_count#27)#50 AS sum(cd_dep_college_count)#62] + +(38) TakeOrderedAndProject +Input [18]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#51, avg(cd_dep_count)#52, max(cd_dep_count)#53, sum(cd_dep_count)#54, cd_dep_employed_count#26, cnt2#55, avg(cd_dep_employed_count)#56, max(cd_dep_employed_count)#57, sum(cd_dep_employed_count)#58, cd_dep_college_count#27, cnt3#59, avg(cd_dep_college_count)#60, max(cd_dep_college_count)#61, sum(cd_dep_college_count)#62] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#51, avg(cd_dep_count)#52, max(cd_dep_count)#53, sum(cd_dep_count)#54, cd_dep_employed_count#26, cnt2#55, avg(cd_dep_employed_count)#56, max(cd_dep_employed_count)#57, sum(cd_dep_employed_count)#58, cd_dep_college_count#27, cnt3#59, avg(cd_dep_college_count)#60, max(cd_dep_college_count)#61, sum(cd_dep_college_count)#62] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) -(41) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#74, d_qoy#75] +(39) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] -Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 1999)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#7)) +(40) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) -(43) CometProject -Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] +(41) CometProject +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) ColumnarToRow [codegen id : 1] +(42) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(45) BroadcastExchange +(43) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt index e5cb940552..e3a91e4713 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt @@ -1,71 +1,52 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (2) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] InputAdapter Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customsk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + CometProject [c_current_cdemo_sk,ca_state] + CometBroadcastHashJoin [c_current_addr_sk,ca_address_sk] + CometProject [c_current_cdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [c_customer_sk,customsk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_current_addr_sk,c_current_cdemo_sk] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_year,d_qoy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [ca_address_sk] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometBroadcastExchange #2 + CometProject [ss_customer_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #4 + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometBroadcastExchange #5 + CometUnion + CometProject [ws_bill_customer_sk] [customsk] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometProject [cs_ship_customer_sk] [customsk] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #6 + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange #7 + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt index 42a213b7cc..3bdded0ed9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt @@ -1,45 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- Window (39) - +- * Sort (38) - +- Exchange (37) - +- * HashAggregate (36) - +- Exchange (35) - +- * HashAggregate (34) - +- Union (33) - :- * HashAggregate (22) - : +- Exchange (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (17) - : +- * ColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.store (13) - :- * HashAggregate (27) - : +- Exchange (26) - : +- * HashAggregate (25) - : +- * HashAggregate (24) - : +- ReusedExchange (23) - +- * HashAggregate (32) - +- Exchange (31) - +- * HashAggregate (30) - +- * HashAggregate (29) - +- ReusedExchange (28) +TakeOrderedAndProject (42) ++- * Project (41) + +- Window (40) + +- * Sort (39) + +- Exchange (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- Union (34) + :- * HashAggregate (23) + : +- Exchange (22) + : +- * ColumnarToRow (21) + : +- CometHashAggregate (20) + : +- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.item (9) + : +- CometBroadcastExchange (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.store (14) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * HashAggregate (25) + : +- ReusedExchange (24) + +- * HashAggregate (33) + +- Exchange (32) + +- * HashAggregate (31) + +- * HashAggregate (30) + +- ReusedExchange (29) (1) Scan parquet spark_catalog.default.store_sales @@ -54,226 +55,231 @@ ReadSchema: struct + +(4) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(4) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#7] +(5) CometProject +Input [2]: [d_date_sk#7, d_year#8] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] +Right output [1]: [d_date_sk#7] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +(8) CometProject Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -(7) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] +(9) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#9, i_class#10, i_category#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) +(10) CometFilter +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Condition : isnotnull(i_item_sk#9) -(9) ColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#9, i_category#10] +(11) CometBroadcastExchange +Input [3]: [i_item_sk#9, i_class#10, i_category#11] +Arguments: [i_item_sk#9, i_class#10, i_category#11] -(10) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(12) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] +Right output [3]: [i_item_sk#9, i_class#10, i_category#11] +Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None +(13) CometProject +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#10, i_category#11] +Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] -(12) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#9, i_category#10] - -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_state#12] +(14) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#12, s_state#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [2]: [s_store_sk#11, s_state#12] -Condition : ((isnotnull(s_state#12) AND (s_state#12 = TN)) AND isnotnull(s_store_sk#11)) - -(15) CometProject -Input [2]: [s_store_sk#11, s_state#12] -Arguments: [s_store_sk#11], [s_store_sk#11] +(15) CometFilter +Input [2]: [s_store_sk#12, s_state#13] +Condition : ((isnotnull(s_state#13) AND (s_state#13 = TN)) AND isnotnull(s_store_sk#12)) -(16) ColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#11] +(16) CometProject +Input [2]: [s_store_sk#12, s_state#13] +Arguments: [s_store_sk#12], [s_store_sk#12] -(17) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(17) CometBroadcastExchange +Input [1]: [s_store_sk#12] +Arguments: [s_store_sk#12] -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(18) CometBroadcastHashJoin +Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] +Right output [1]: [s_store_sk#12] +Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight -(19) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10, s_store_sk#11] +(19) CometProject +Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11, s_store_sk#12] +Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11], [ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] -(20) HashAggregate [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] -Keys [2]: [i_category#10, i_class#9] +(20) CometHashAggregate +Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] +Keys [2]: [i_category#11, i_class#10] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#13, sum#14] -Results [4]: [i_category#10, i_class#9, sum#15, sum#16] -(21) Exchange -Input [4]: [i_category#10, i_class#9, sum#15, sum#16] -Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) ColumnarToRow [codegen id : 1] +Input [4]: [i_category#11, i_class#10, sum#14, sum#15] + +(22) Exchange +Input [4]: [i_category#11, i_class#10, sum#14, sum#15] +Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(22) HashAggregate [codegen id : 5] -Input [4]: [i_category#10, i_class#9, sum#15, sum#16] -Keys [2]: [i_category#10, i_class#9] +(23) HashAggregate [codegen id : 2] +Input [4]: [i_category#11, i_class#10, sum#14, sum#15] +Keys [2]: [i_category#11, i_class#10] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#17, sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#17,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2)) as decimal(38,20)) AS gross_margin#19, i_category#10, i_class#9, 0 AS t_category#20, 0 AS t_class#21, 0 AS lochierarchy#22] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#16, sum(UnscaledValue(ss_ext_sales_price#3))#17] +Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#16,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2)) as decimal(38,20)) AS gross_margin#18, i_category#11, i_class#10, 0 AS t_category#19, 0 AS t_class#20, 0 AS lochierarchy#21] -(23) ReusedExchange [Reuses operator id: 21] -Output [4]: [i_category#10, i_class#9, sum#23, sum#24] +(24) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#11, i_class#10, sum#22, sum#23] -(24) HashAggregate [codegen id : 10] -Input [4]: [i_category#10, i_class#9, sum#23, sum#24] -Keys [2]: [i_category#10, i_class#9] +(25) HashAggregate [codegen id : 4] +Input [4]: [i_category#11, i_class#10, sum#22, sum#23] +Keys [2]: [i_category#11, i_class#10] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#25, sum(UnscaledValue(ss_ext_sales_price#3))#26] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#25,17,2) AS ss_net_profit#27, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#26,17,2) AS ss_ext_sales_price#28, i_category#10] - -(25) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#27, ss_ext_sales_price#28, i_category#10] -Keys [1]: [i_category#10] -Functions [2]: [partial_sum(ss_net_profit#27), partial_sum(ss_ext_sales_price#28)] -Aggregate Attributes [4]: [sum#29, isEmpty#30, sum#31, isEmpty#32] -Results [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] - -(26) Exchange -Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(27) HashAggregate [codegen id : 11] -Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36] -Keys [1]: [i_category#10] -Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)] -Aggregate Attributes [2]: [sum(ss_net_profit#27)#37, sum(ss_ext_sales_price#28)#38] -Results [6]: [cast((sum(ss_net_profit#27)#37 / sum(ss_ext_sales_price#28)#38) as decimal(38,20)) AS gross_margin#39, i_category#10, null AS i_class#40, 0 AS t_category#41, 1 AS t_class#42, 1 AS lochierarchy#43] - -(28) ReusedExchange [Reuses operator id: 21] -Output [4]: [i_category#10, i_class#9, sum#44, sum#45] - -(29) HashAggregate [codegen id : 16] -Input [4]: [i_category#10, i_class#9, sum#44, sum#45] -Keys [2]: [i_category#10, i_class#9] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#24, sum(UnscaledValue(ss_ext_sales_price#3))#25] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#24,17,2) AS ss_net_profit#26, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#25,17,2) AS ss_ext_sales_price#27, i_category#11] + +(26) HashAggregate [codegen id : 4] +Input [3]: [ss_net_profit#26, ss_ext_sales_price#27, i_category#11] +Keys [1]: [i_category#11] +Functions [2]: [partial_sum(ss_net_profit#26), partial_sum(ss_ext_sales_price#27)] +Aggregate Attributes [4]: [sum#28, isEmpty#29, sum#30, isEmpty#31] +Results [5]: [i_category#11, sum#32, isEmpty#33, sum#34, isEmpty#35] + +(27) Exchange +Input [5]: [i_category#11, sum#32, isEmpty#33, sum#34, isEmpty#35] +Arguments: hashpartitioning(i_category#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(28) HashAggregate [codegen id : 5] +Input [5]: [i_category#11, sum#32, isEmpty#33, sum#34, isEmpty#35] +Keys [1]: [i_category#11] +Functions [2]: [sum(ss_net_profit#26), sum(ss_ext_sales_price#27)] +Aggregate Attributes [2]: [sum(ss_net_profit#26)#36, sum(ss_ext_sales_price#27)#37] +Results [6]: [cast((sum(ss_net_profit#26)#36 / sum(ss_ext_sales_price#27)#37) as decimal(38,20)) AS gross_margin#38, i_category#11, null AS i_class#39, 0 AS t_category#40, 1 AS t_class#41, 1 AS lochierarchy#42] + +(29) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#11, i_class#10, sum#43, sum#44] + +(30) HashAggregate [codegen id : 7] +Input [4]: [i_category#11, i_class#10, sum#43, sum#44] +Keys [2]: [i_category#11, i_class#10] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#25, sum(UnscaledValue(ss_ext_sales_price#3))#26] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#25,17,2) AS ss_net_profit#27, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#26,17,2) AS ss_ext_sales_price#28] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#24, sum(UnscaledValue(ss_ext_sales_price#3))#25] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#24,17,2) AS ss_net_profit#26, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#25,17,2) AS ss_ext_sales_price#27] -(30) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#27, ss_ext_sales_price#28] +(31) HashAggregate [codegen id : 7] +Input [2]: [ss_net_profit#26, ss_ext_sales_price#27] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#27), partial_sum(ss_ext_sales_price#28)] -Aggregate Attributes [4]: [sum#46, isEmpty#47, sum#48, isEmpty#49] -Results [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] +Functions [2]: [partial_sum(ss_net_profit#26), partial_sum(ss_ext_sales_price#27)] +Aggregate Attributes [4]: [sum#45, isEmpty#46, sum#47, isEmpty#48] +Results [4]: [sum#49, isEmpty#50, sum#51, isEmpty#52] -(31) Exchange -Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] +(32) Exchange +Input [4]: [sum#49, isEmpty#50, sum#51, isEmpty#52] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(32) HashAggregate [codegen id : 17] -Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53] +(33) HashAggregate [codegen id : 8] +Input [4]: [sum#49, isEmpty#50, sum#51, isEmpty#52] Keys: [] -Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)] -Aggregate Attributes [2]: [sum(ss_net_profit#27)#54, sum(ss_ext_sales_price#28)#55] -Results [6]: [cast((sum(ss_net_profit#27)#54 / sum(ss_ext_sales_price#28)#55) as decimal(38,20)) AS gross_margin#56, null AS i_category#57, null AS i_class#58, 1 AS t_category#59, 1 AS t_class#60, 2 AS lochierarchy#61] +Functions [2]: [sum(ss_net_profit#26), sum(ss_ext_sales_price#27)] +Aggregate Attributes [2]: [sum(ss_net_profit#26)#53, sum(ss_ext_sales_price#27)#54] +Results [6]: [cast((sum(ss_net_profit#26)#53 / sum(ss_ext_sales_price#27)#54) as decimal(38,20)) AS gross_margin#55, null AS i_category#56, null AS i_class#57, 1 AS t_category#58, 1 AS t_class#59, 2 AS lochierarchy#60] -(33) Union +(34) Union -(34) HashAggregate [codegen id : 18] -Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +(35) HashAggregate [codegen id : 9] +Input [6]: [gross_margin#18, i_category#11, i_class#10, t_category#19, t_class#20, lochierarchy#21] +Keys [6]: [gross_margin#18, i_category#11, i_class#10, t_category#19, t_class#20, lochierarchy#21] Functions: [] Aggregate Attributes: [] -Results [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +Results [6]: [gross_margin#18, i_category#11, i_class#10, t_category#19, t_class#20, lochierarchy#21] -(35) Exchange -Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -Arguments: hashpartitioning(gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(36) Exchange +Input [6]: [gross_margin#18, i_category#11, i_class#10, t_category#19, t_class#20, lochierarchy#21] +Arguments: hashpartitioning(gross_margin#18, i_category#11, i_class#10, t_category#19, t_class#20, lochierarchy#21, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(36) HashAggregate [codegen id : 19] -Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] -Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +(37) HashAggregate [codegen id : 10] +Input [6]: [gross_margin#18, i_category#11, i_class#10, t_category#19, t_class#20, lochierarchy#21] +Keys [6]: [gross_margin#18, i_category#11, i_class#10, t_category#19, t_class#20, lochierarchy#21] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, CASE WHEN (t_class#21 = 0) THEN i_category#10 END AS _w0#62] +Results [5]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, CASE WHEN (t_class#20 = 0) THEN i_category#11 END AS _w0#61] -(37) Exchange -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] -Arguments: hashpartitioning(lochierarchy#22, _w0#62, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(38) Exchange +Input [5]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, _w0#61] +Arguments: hashpartitioning(lochierarchy#21, _w0#61, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(38) Sort [codegen id : 20] -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] -Arguments: [lochierarchy#22 ASC NULLS FIRST, _w0#62 ASC NULLS FIRST, gross_margin#19 ASC NULLS FIRST], false, 0 +(39) Sort [codegen id : 11] +Input [5]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, _w0#61] +Arguments: [lochierarchy#21 ASC NULLS FIRST, _w0#61 ASC NULLS FIRST, gross_margin#18 ASC NULLS FIRST], false, 0 -(39) Window -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62] -Arguments: [rank(gross_margin#19) windowspecdefinition(lochierarchy#22, _w0#62, gross_margin#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#63], [lochierarchy#22, _w0#62], [gross_margin#19 ASC NULLS FIRST] +(40) Window +Input [5]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, _w0#61] +Arguments: [rank(gross_margin#18) windowspecdefinition(lochierarchy#21, _w0#61, gross_margin#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#62], [lochierarchy#21, _w0#61], [gross_margin#18 ASC NULLS FIRST] -(40) Project [codegen id : 21] -Output [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] -Input [6]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#62, rank_within_parent#63] +(41) Project [codegen id : 12] +Output [5]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, rank_within_parent#62] +Input [6]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, _w0#61, rank_within_parent#62] -(41) TakeOrderedAndProject -Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] -Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#63] +(42) TakeOrderedAndProject +Input [5]: [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, rank_within_parent#62] +Arguments: 100, [lochierarchy#21 DESC NULLS LAST, CASE WHEN (lochierarchy#21 = 0) THEN i_category#11 END ASC NULLS FIRST, rank_within_parent#62 ASC NULLS FIRST], [gross_margin#18, i_category#11, i_class#10, lochierarchy#21, rank_within_parent#62] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (46) -+- * ColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.date_dim (42) +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(42) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#64] +(43) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_year#8] 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 -(43) CometFilter -Input [2]: [d_date_sk#7, d_year#64] -Condition : ((isnotnull(d_year#64) AND (d_year#64 = 2001)) AND isnotnull(d_date_sk#7)) +(44) CometFilter +Input [2]: [d_date_sk#7, d_year#8] +Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) -(44) CometProject -Input [2]: [d_date_sk#7, d_year#64] +(45) CometProject +Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(45) ColumnarToRow [codegen id : 1] +(46) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(46) BroadcastExchange +(47) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index f265d20995..9c35ee397c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -1,34 +1,34 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (21) + WholeStageCodegen (12) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (20) + WholeStageCodegen (11) Sort [lochierarchy,_w0,gross_margin] InputAdapter Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (19) + WholeStageCodegen (10) HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] [_w0] InputAdapter Exchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (18) + WholeStageCodegen (9) HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,t_category,t_class,lochierarchy,sum,sum] InputAdapter Exchange [i_category,i_class] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk,ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -39,37 +39,31 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometProject [d_date_sk] CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_state,s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (11) + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange #7 + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [i_category] #7 - WholeStageCodegen (10) + Exchange [i_category] #8 + WholeStageCodegen (4) HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange #8 - WholeStageCodegen (16) + Exchange #9 + WholeStageCodegen (7) HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index 727f50abf0..41d045790e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -13,23 +13,23 @@ TakeOrderedAndProject (45) : : +- Exchange (22) : : +- * HashAggregate (21) : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) + : : +- * ColumnarToRow (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.store (13) : +- BroadcastExchange (35) @@ -57,10 +57,7 @@ ReadSchema: struct Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] - -(4) Scan parquet spark_catalog.default.store_sales +(3) Scan parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -68,39 +65,46 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] Condition : (isnotnull(ss_item_sk#4) AND isnotnull(ss_store_sk#5)) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +Arguments: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#4] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Right output [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: [i_item_sk#1], [ss_item_sk#4], Inner, BuildRight -(9) Project [codegen id : 4] -Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +(7) CometProject Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7], [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_year#10, d_moy#11] + +(11) CometBroadcastHashJoin +Left output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Right output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(12) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] +(12) CometProject Input [8]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] +Arguments: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11], [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] (13) Scan parquet spark_catalog.default.store Output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] @@ -113,145 +117,142 @@ ReadSchema: struct Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] Condition : ((isnotnull(s_store_sk#12) AND isnotnull(s_store_name#13)) AND isnotnull(s_company_name#14)) -(15) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [s_store_sk#12, s_store_name#13, s_company_name#14] -(16) BroadcastExchange -Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] +Right output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [ss_store_sk#5], [s_store_sk#12], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] +(17) CometProject Input [9]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14], [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] -(19) HashAggregate [codegen id : 4] +(18) CometHashAggregate Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [1]: [sum#15] -Results [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] + +(19) ColumnarToRow [codegen id : 1] +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#15] (20) Exchange -Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#15] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] +(21) HashAggregate [codegen id : 2] +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#15] Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#17] -Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS _w0#19] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#16] +Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#16,17,2) AS _w0#18] (22) Exchange -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(23) Sort [codegen id : 6] -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +(23) Sort [codegen id : 3] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 (24) Window -Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] +(25) Filter [codegen id : 4] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) (26) Window -Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] -(27) Filter [codegen id : 22] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) +(27) Filter [codegen id : 13] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(28) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +(28) Project [codegen id : 13] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] (29) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Output [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] -(30) HashAggregate [codegen id : 12] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] -Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#18] +(30) HashAggregate [codegen id : 6] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum#27] +Keys [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26] +Functions [1]: [sum(UnscaledValue(ss_sales_price#28))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#28))#16] +Results [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(ss_sales_price#28))#16,17,2) AS sum_sales#17] (31) Exchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: hashpartitioning(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 13] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 +(32) Sort [codegen id : 7] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, s_store_name#23 ASC NULLS FIRST, s_company_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST], false, 0 (33) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] +Input [7]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17] +Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, s_store_name#23, s_company_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(34) Project [codegen id : 14] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#18 AS sum_sales#31, rn#30] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#18, rn#30] +(34) Project [codegen id : 8] +Output [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#17 AS sum_sales#30, rn#29] +Input [8]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, d_year#25, d_moy#26, sum_sales#17, rn#29] (35) BroadcastExchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] +Input [6]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#30, rn#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#30 + 1)] +(36) BroadcastHashJoin [codegen id : 13] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] +Right keys [5]: [i_category#21, i_brand#22, s_store_name#23, s_company_name#24, (rn#29 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] -Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#31, rn#30] +(37) Project [codegen id : 13] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#30] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, s_store_name#23, s_company_name#24, sum_sales#30, rn#29] (38) ReusedExchange [Reuses operator id: 31] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] +Output [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] -(39) Sort [codegen id : 20] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 +(39) Sort [codegen id : 11] +Input [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] +Arguments: [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, s_store_name#33 ASC NULLS FIRST, s_company_name#34 ASC NULLS FIRST, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST], false, 0 (40) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] +Input [7]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17] +Arguments: [rank(d_year#35, d_moy#36) windowspecdefinition(i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, s_store_name#33, s_company_name#34], [d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] -(41) Project [codegen id : 21] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#18 AS sum_sales#39, rn#38] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#18, rn#38] +(41) Project [codegen id : 12] +Output [6]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#17 AS sum_sales#38, rn#37] +Input [8]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, d_year#35, d_moy#36, sum_sales#17, rn#37] (42) BroadcastExchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] +Input [6]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#38, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#38 - 1)] +(43) BroadcastHashJoin [codegen id : 13] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#19] +Right keys [5]: [i_category#31, i_brand#32, s_store_name#33, s_company_name#34, (rn#37 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] -Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#39, rn#38] +(44) Project [codegen id : 13] +Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#30 AS psum#39, sum_sales#38 AS nsum#40] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#30, i_category#31, i_brand#32, s_store_name#33, s_company_name#34, sum_sales#38, rn#37] (45) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#39, nsum#40] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#39, nsum#40] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (49) +- * ColumnarToRow (48) +- CometFilter (47) @@ -274,6 +275,6 @@ Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index a548953052..a85302cc57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,74 +8,67 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter Exchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,ss_item_sk] CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_store_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange #3 + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) + BroadcastExchange #7 + WholeStageCodegen (8) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) + WholeStageCodegen (7) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) + Exchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (6) HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] InputAdapter ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #9 + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) + WholeStageCodegen (11) Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index 6591c8b8a2..07bd1d0ca0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -1,78 +1,81 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- Exchange (72) - +- * HashAggregate (71) - +- Union (70) - :- * Project (23) - : +- * Filter (22) - : +- Window (21) - : +- * Sort (20) - : +- Window (19) - : +- * Sort (18) - : +- Exchange (17) - : +- * HashAggregate (16) - : +- Exchange (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * ColumnarToRow (10) - : : +- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan parquet spark_catalog.default.web_returns (5) - : +- ReusedExchange (11) - :- * Project (46) - : +- * Filter (45) - : +- Window (44) - : +- * Sort (43) - : +- Window (42) - : +- * Sort (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * ColumnarToRow (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometScan parquet spark_catalog.default.catalog_returns (28) - : +- ReusedExchange (34) - +- * Project (69) - +- * Filter (68) - +- Window (67) - +- * Sort (66) - +- Window (65) - +- * Sort (64) - +- Exchange (63) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * ColumnarToRow (56) - : +- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometBroadcastExchange (50) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan parquet spark_catalog.default.store_sales (47) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.store_returns (51) - +- ReusedExchange (57) +TakeOrderedAndProject (77) ++- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * Sort (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- Exchange (18) + : +- * ColumnarToRow (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * Sort (44) + : +- Exchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * ColumnarToRow (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * Sort (67) + +- Exchange (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * ColumnarToRow (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) Scan parquet spark_catalog.default.web_sales @@ -113,344 +116,351 @@ Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_am (8) CometBroadcastHashJoin Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner +Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft (9) CometProject Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -(10) ColumnarToRow [codegen id : 2] -Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +(10) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#13] +(11) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) + +(12) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Arguments: [d_date_sk#13], [d_date_sk#13] + +(13) CometBroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: [d_date_sk#13] -(12) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(14) CometBroadcastHashJoin +Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] +Right output [1]: [d_date_sk#13] +Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight -(13) Project [codegen id : 2] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] +(15) CometProject Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(14) HashAggregate [codegen id : 2] +(16) CometHashAggregate Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(15) Exchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(17) ColumnarToRow [codegen id : 1] +Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] + +(18) Exchange +Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(16) HashAggregate [codegen id : 3] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +(19) HashAggregate [codegen id : 2] +Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] +Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#22, sum(coalesce(ws_quantity#3, 0))#23, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#24, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#25] +Results [3]: [ws_item_sk#1 AS item#26, (cast(sum(coalesce(wr_return_quantity#10, 0))#22 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#23 as decimal(15,4))) AS return_ratio#27, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#24 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#25 as decimal(15,4))) AS currency_ratio#28] -(17) Exchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] +(20) Exchange +Input [3]: [item#26, return_ratio#27, currency_ratio#28] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(18) Sort [codegen id : 4] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 +(21) Sort [codegen id : 3] +Input [3]: [item#26, return_ratio#27, currency_ratio#28] +Arguments: [return_ratio#27 ASC NULLS FIRST], false, 0 -(19) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] +(22) Window +Input [3]: [item#26, return_ratio#27, currency_ratio#28] +Arguments: [rank(return_ratio#27) windowspecdefinition(return_ratio#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#29], [return_ratio#27 ASC NULLS FIRST] -(20) Sort [codegen id : 5] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 4] +Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] +Arguments: [currency_ratio#28 ASC NULLS FIRST], false, 0 -(21) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] +(24) Window +Input [4]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29] +Arguments: [rank(currency_ratio#28) windowspecdefinition(currency_ratio#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#30], [currency_ratio#28 ASC NULLS FIRST] -(22) Filter [codegen id : 6] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) +(25) Filter [codegen id : 5] +Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] +Condition : ((return_rank#29 <= 10) OR (currency_rank#30 <= 10)) -(23) Project [codegen id : 6] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +(26) Project [codegen id : 5] +Output [5]: [web AS channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Input [5]: [item#26, return_ratio#27, currency_ratio#28, return_rank#29, currency_rank#30] -(24) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +(27) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(25) CometFilter -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) +(28) CometFilter +Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] +Condition : (((((((isnotnull(cs_net_profit#36) AND isnotnull(cs_net_paid#35)) AND isnotnull(cs_quantity#34)) AND (cs_net_profit#36 > 1.00)) AND (cs_net_paid#35 > 0.00)) AND (cs_quantity#34 > 0)) AND isnotnull(cs_order_number#33)) AND isnotnull(cs_item_sk#32)) -(26) CometProject -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(29) CometProject +Input [6]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_net_profit#36, cs_sold_date_sk#37] +Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37], [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] -(27) CometBroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +(30) CometBroadcastExchange +Input [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +Arguments: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] -(28) Scan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +(31) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(29) CometFilter -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) - -(30) CometProject -Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] -Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] - -(31) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner - -(32) CometProject -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] -Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] - -(33) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] - -(34) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#48] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] - -(37) HashAggregate [codegen id : 8] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] -Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] - -(38) Exchange -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(39) HashAggregate [codegen id : 9] -Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] -Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] - -(40) Exchange -Input [3]: [item#65, return_ratio#66, currency_ratio#67] +(32) CometFilter +Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] +Condition : (((isnotnull(cr_return_amount#42) AND (cr_return_amount#42 > 10000.00)) AND isnotnull(cr_order_number#40)) AND isnotnull(cr_item_sk#39)) + +(33) CometProject +Input [5]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42, cr_returned_date_sk#43] +Arguments: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42], [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] + +(34) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37] +Right output [4]: [cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] +Arguments: [cs_order_number#33, cs_item_sk#32], [cr_order_number#40, cr_item_sk#39], Inner, BuildLeft + +(35) CometProject +Input [9]: [cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40, cr_return_quantity#41, cr_return_amount#42] +Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] + +(36) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#44] + +(37) CometBroadcastHashJoin +Left output [6]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42] +Right output [1]: [d_date_sk#44] +Arguments: [cs_sold_date_sk#37], [d_date_sk#44], Inner, BuildRight + +(38) CometProject +Input [7]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cs_sold_date_sk#37, cr_return_quantity#41, cr_return_amount#42, d_date_sk#44] +Arguments: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42], [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] + +(39) CometHashAggregate +Input [5]: [cs_item_sk#32, cs_quantity#34, cs_net_paid#35, cr_return_quantity#41, cr_return_amount#42] +Keys [1]: [cs_item_sk#32] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#41, 0)), partial_sum(coalesce(cs_quantity#34, 0)), partial_sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] + +(40) ColumnarToRow [codegen id : 6] +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] + +(41) Exchange +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] +Arguments: hashpartitioning(cs_item_sk#32, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(42) HashAggregate [codegen id : 7] +Input [7]: [cs_item_sk#32, sum#45, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] +Keys [1]: [cs_item_sk#32] +Functions [4]: [sum(coalesce(cr_return_quantity#41, 0)), sum(coalesce(cs_quantity#34, 0)), sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#41, 0))#51, sum(coalesce(cs_quantity#34, 0))#52, sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#53, sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#54] +Results [3]: [cs_item_sk#32 AS item#55, (cast(sum(coalesce(cr_return_quantity#41, 0))#51 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#34, 0))#52 as decimal(15,4))) AS return_ratio#56, (cast(sum(coalesce(cast(cr_return_amount#42 as decimal(12,2)), 0.00))#53 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#35 as decimal(12,2)), 0.00))#54 as decimal(15,4))) AS currency_ratio#57] + +(43) Exchange +Input [3]: [item#55, return_ratio#56, currency_ratio#57] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(41) Sort [codegen id : 10] -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 +(44) Sort [codegen id : 8] +Input [3]: [item#55, return_ratio#56, currency_ratio#57] +Arguments: [return_ratio#56 ASC NULLS FIRST], false, 0 -(42) Window -Input [3]: [item#65, return_ratio#66, currency_ratio#67] -Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] +(45) Window +Input [3]: [item#55, return_ratio#56, currency_ratio#57] +Arguments: [rank(return_ratio#56) windowspecdefinition(return_ratio#56 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#58], [return_ratio#56 ASC NULLS FIRST] -(43) Sort [codegen id : 11] -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 9] +Input [4]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58] +Arguments: [currency_ratio#57 ASC NULLS FIRST], false, 0 -(44) Window -Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] -Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] +(47) Window +Input [4]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58] +Arguments: [rank(currency_ratio#57) windowspecdefinition(currency_ratio#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#59], [currency_ratio#57 ASC NULLS FIRST] -(45) Filter [codegen id : 12] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] -Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) +(48) Filter [codegen id : 10] +Input [5]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58, currency_rank#59] +Condition : ((return_rank#58 <= 10) OR (currency_rank#59 <= 10)) -(46) Project [codegen id : 12] -Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] -Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +(49) Project [codegen id : 10] +Output [5]: [catalog AS channel#60, item#55, return_ratio#56, return_rank#58, currency_rank#59] +Input [5]: [item#55, return_ratio#56, currency_ratio#57, return_rank#58, currency_rank#59] -(47) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +(50) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(48) CometFilter -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) +(51) CometFilter +Input [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] +Condition : (((((((isnotnull(ss_net_profit#65) AND isnotnull(ss_net_paid#64)) AND isnotnull(ss_quantity#63)) AND (ss_net_profit#65 > 1.00)) AND (ss_net_paid#64 > 0.00)) AND (ss_quantity#63 > 0)) AND isnotnull(ss_ticket_number#62)) AND isnotnull(ss_item_sk#61)) -(49) CometProject -Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(52) CometProject +Input [6]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_net_profit#65, ss_sold_date_sk#66] +Arguments: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66], [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] -(50) CometBroadcastExchange -Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +(53) CometBroadcastExchange +Input [5]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] +Arguments: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] -(51) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +(54) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(52) CometFilter -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) - -(53) CometProject -Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] -Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] - -(54) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] -Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner - -(55) CometProject -Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] -Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] - -(56) ColumnarToRow [codegen id : 14] -Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] - -(57) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#83] - -(58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#76] -Right keys [1]: [d_date_sk#83] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 14] -Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] - -(60) HashAggregate [codegen id : 14] -Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] -Keys [1]: [ss_item_sk#71] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] -Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] - -(61) Exchange -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(62) HashAggregate [codegen id : 15] -Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Keys [1]: [ss_item_sk#71] -Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] -Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] - -(63) Exchange -Input [3]: [item#100, return_ratio#101, currency_ratio#102] +(55) CometFilter +Input [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] +Condition : (((isnotnull(sr_return_amt#71) AND (sr_return_amt#71 > 10000.00)) AND isnotnull(sr_ticket_number#69)) AND isnotnull(sr_item_sk#68)) + +(56) CometProject +Input [5]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71, sr_returned_date_sk#72] +Arguments: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71], [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] + +(57) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66] +Right output [4]: [sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] +Arguments: [ss_ticket_number#62, ss_item_sk#61], [sr_ticket_number#69, sr_item_sk#68], Inner, BuildLeft + +(58) CometProject +Input [9]: [ss_item_sk#61, ss_ticket_number#62, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_item_sk#68, sr_ticket_number#69, sr_return_quantity#70, sr_return_amt#71] +Arguments: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71], [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71] + +(59) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_date_sk#73] + +(60) CometBroadcastHashJoin +Left output [6]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71] +Right output [1]: [d_date_sk#73] +Arguments: [ss_sold_date_sk#66], [d_date_sk#73], Inner, BuildRight + +(61) CometProject +Input [7]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, ss_sold_date_sk#66, sr_return_quantity#70, sr_return_amt#71, d_date_sk#73] +Arguments: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71], [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71] + +(62) CometHashAggregate +Input [5]: [ss_item_sk#61, ss_quantity#63, ss_net_paid#64, sr_return_quantity#70, sr_return_amt#71] +Keys [1]: [ss_item_sk#61] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#70, 0)), partial_sum(coalesce(ss_quantity#63, 0)), partial_sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))] + +(63) ColumnarToRow [codegen id : 11] +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] + +(64) Exchange +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] +Arguments: hashpartitioning(ss_item_sk#61, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(65) HashAggregate [codegen id : 12] +Input [7]: [ss_item_sk#61, sum#74, sum#75, sum#76, isEmpty#77, sum#78, isEmpty#79] +Keys [1]: [ss_item_sk#61] +Functions [4]: [sum(coalesce(sr_return_quantity#70, 0)), sum(coalesce(ss_quantity#63, 0)), sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#70, 0))#80, sum(coalesce(ss_quantity#63, 0))#81, sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00))#82, sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))#83] +Results [3]: [ss_item_sk#61 AS item#84, (cast(sum(coalesce(sr_return_quantity#70, 0))#80 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#63, 0))#81 as decimal(15,4))) AS return_ratio#85, (cast(sum(coalesce(cast(sr_return_amt#71 as decimal(12,2)), 0.00))#82 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#64 as decimal(12,2)), 0.00))#83 as decimal(15,4))) AS currency_ratio#86] + +(66) Exchange +Input [3]: [item#84, return_ratio#85, currency_ratio#86] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(64) Sort [codegen id : 16] -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 +(67) Sort [codegen id : 13] +Input [3]: [item#84, return_ratio#85, currency_ratio#86] +Arguments: [return_ratio#85 ASC NULLS FIRST], false, 0 -(65) Window -Input [3]: [item#100, return_ratio#101, currency_ratio#102] -Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] +(68) Window +Input [3]: [item#84, return_ratio#85, currency_ratio#86] +Arguments: [rank(return_ratio#85) windowspecdefinition(return_ratio#85 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#87], [return_ratio#85 ASC NULLS FIRST] -(66) Sort [codegen id : 17] -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 +(69) Sort [codegen id : 14] +Input [4]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87] +Arguments: [currency_ratio#86 ASC NULLS FIRST], false, 0 -(67) Window -Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] -Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] +(70) Window +Input [4]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87] +Arguments: [rank(currency_ratio#86) windowspecdefinition(currency_ratio#86 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#88], [currency_ratio#86 ASC NULLS FIRST] -(68) Filter [codegen id : 18] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] -Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) +(71) Filter [codegen id : 15] +Input [5]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87, currency_rank#88] +Condition : ((return_rank#87 <= 10) OR (currency_rank#88 <= 10)) -(69) Project [codegen id : 18] -Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] -Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] +(72) Project [codegen id : 15] +Output [5]: [store AS channel#89, item#84, return_ratio#85, return_rank#87, currency_rank#88] +Input [5]: [item#84, return_ratio#85, currency_ratio#86, return_rank#87, currency_rank#88] -(70) Union +(73) Union -(71) HashAggregate [codegen id : 19] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(74) HashAggregate [codegen id : 16] +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(72) Exchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] +(75) Exchange +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Arguments: hashpartitioning(channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(73) HashAggregate [codegen id : 20] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(76) HashAggregate [codegen id : 17] +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Keys [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Results [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] -(74) TakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +(77) TakeOrderedAndProject +Input [5]: [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] +Arguments: 100, [channel#31 ASC NULLS FIRST, return_rank#29 ASC NULLS FIRST, currency_rank#30 ASC NULLS FIRST, item#26 ASC NULLS FIRST], [channel#31, item#26, return_ratio#27, return_rank#29, currency_rank#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (79) -+- * ColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan parquet spark_catalog.default.date_dim (75) +BroadcastExchange (82) ++- * ColumnarToRow (81) + +- CometProject (80) + +- CometFilter (79) + +- CometScan parquet spark_catalog.default.date_dim (78) -(75) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#106, d_moy#107] +(78) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] -Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) +(79) CometFilter +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(77) CometProject -Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +(80) CometProject +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(78) ColumnarToRow [codegen id : 1] +(81) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(79) BroadcastExchange +(82) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index 43ebf34cc0..8d7b158d0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -1,35 +1,35 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (20) + WholeStageCodegen (17) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (19) + WholeStageCodegen (16) HashAggregate [channel,item,return_ratio,return_rank,currency_rank] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (5) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (5) + WholeStageCodegen (4) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (4) + WholeStageCodegen (3) Sort [return_ratio] InputAdapter Exchange #2 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [ws_item_sk] #3 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] CometBroadcastExchange #4 @@ -47,34 +47,36 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_return_amt,wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (12) + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (10) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (11) + WholeStageCodegen (9) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (10) + WholeStageCodegen (8) Sort [return_ratio] InputAdapter - Exchange #6 - WholeStageCodegen (9) + Exchange #7 + WholeStageCodegen (7) HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [cs_item_sk] #7 - WholeStageCodegen (8) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [cs_item_sk] #8 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - CometBroadcastExchange #8 + CometBroadcastExchange #9 CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] @@ -82,34 +84,33 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_return_amount,cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (18) + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (15) Project [item,return_ratio,return_rank,currency_rank] Filter [return_rank,currency_rank] InputAdapter Window [currency_ratio] - WholeStageCodegen (17) + WholeStageCodegen (14) Sort [currency_ratio] InputAdapter Window [return_ratio] - WholeStageCodegen (16) + WholeStageCodegen (13) Sort [return_ratio] InputAdapter - Exchange #9 - WholeStageCodegen (15) + Exchange #10 + WholeStageCodegen (12) HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [ss_item_sk] #11 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - CometBroadcastExchange #11 + CometBroadcastExchange #12 CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] @@ -117,5 +118,4 @@ TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index ee61d214af..1bcaa1ea17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -1,71 +1,74 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Filter (66) - +- * HashAggregate (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- Window (56) - : +- * Sort (55) - : +- Exchange (54) - : +- * Project (53) - : +- * Filter (52) - : +- * SortMergeJoin FullOuter (51) - : :- * Sort (25) - : : +- Exchange (24) - : : +- * HashAggregate (23) - : : +- Exchange (22) - : : +- * HashAggregate (21) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (13) - : : : +- Window (12) - : : : +- * Sort (11) - : : : +- Exchange (10) - : : : +- * HashAggregate (9) - : : : +- Exchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (18) - : : +- * Project (17) - : : +- Window (16) - : : +- * Sort (15) - : : +- ReusedExchange (14) - : +- * Sort (50) - : +- Exchange (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (38) - : : +- Window (37) - : : +- * Sort (36) - : : +- Exchange (35) - : : +- * HashAggregate (34) - : : +- Exchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * ColumnarToRow (28) - : : : +- CometFilter (27) - : : : +- CometScan parquet spark_catalog.default.store_sales (26) - : : +- ReusedExchange (29) - : +- BroadcastExchange (43) - : +- * Project (42) - : +- Window (41) - : +- * Sort (40) - : +- ReusedExchange (39) - +- BroadcastExchange (61) - +- * Project (60) - +- Window (59) - +- * Sort (58) - +- ReusedExchange (57) +TakeOrderedAndProject (70) ++- * Filter (69) + +- * HashAggregate (68) + +- * HashAggregate (67) + +- * Project (66) + +- * BroadcastHashJoin Inner BuildRight (65) + :- Window (59) + : +- * Sort (58) + : +- Exchange (57) + : +- * Project (56) + : +- * Filter (55) + : +- * SortMergeJoin FullOuter (54) + : :- * Sort (28) + : : +- Exchange (27) + : : +- * HashAggregate (26) + : : +- Exchange (25) + : : +- * HashAggregate (24) + : : +- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Project (16) + : : : +- Window (15) + : : : +- * Sort (14) + : : : +- Exchange (13) + : : : +- * HashAggregate (12) + : : : +- Exchange (11) + : : : +- * ColumnarToRow (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (21) + : : +- * Project (20) + : : +- Window (19) + : : +- * Sort (18) + : : +- ReusedExchange (17) + : +- * Sort (53) + : +- Exchange (52) + : +- * HashAggregate (51) + : +- Exchange (50) + : +- * HashAggregate (49) + : +- * Project (48) + : +- * BroadcastHashJoin Inner BuildRight (47) + : :- * Project (41) + : : +- Window (40) + : : +- * Sort (39) + : : +- Exchange (38) + : : +- * HashAggregate (37) + : : +- Exchange (36) + : : +- * ColumnarToRow (35) + : : +- CometHashAggregate (34) + : : +- CometProject (33) + : : +- CometBroadcastHashJoin (32) + : : :- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.store_sales (29) + : : +- ReusedExchange (31) + : +- BroadcastExchange (46) + : +- * Project (45) + : +- Window (44) + : +- * Sort (43) + : +- ReusedExchange (42) + +- BroadcastExchange (64) + +- * Project (63) + +- Window (62) + +- * Sort (61) + +- ReusedExchange (60) (1) Scan parquet spark_catalog.default.web_sales @@ -80,112 +83,125 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 72] -Output [2]: [d_date_sk#5, d_date#6] +(4) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] + +(6) CometBroadcastExchange +Input [2]: [d_date_sk#5, d_date#6] +Arguments: [d_date_sk#5, d_date#6] -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +Right output [2]: [d_date_sk#5, d_date#6] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] +Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] -(7) HashAggregate [codegen id : 2] +(9) CometHashAggregate Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] -(8) Exchange +(10) ColumnarToRow [codegen id : 1] +Input [3]: [ws_item_sk#1, d_date#6, sum#8] + +(11) Exchange Input [3]: [ws_item_sk#1, d_date#6, sum#8] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(9) HashAggregate [codegen id : 3] +(12) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] -(10) Exchange +(13) Exchange Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(11) Sort [codegen id : 4] +(14) Sort [codegen id : 3] Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(12) Window +(15) Window Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(13) Project [codegen id : 10] +(16) Project [codegen id : 8] Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] -(14) ReusedExchange [Reuses operator id: 10] +(17) ReusedExchange [Reuses operator id: 13] Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -(15) Sort [codegen id : 8] +(18) Sort [codegen id : 6] Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] Arguments: [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST], false, 0 -(16) Window +(19) Window Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] -(17) Project [codegen id : 9] +(20) Project [codegen id : 7] Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] -(18) BroadcastExchange +(21) BroadcastExchange Input [3]: [item_sk#16, sumws#17, rk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(19) BroadcastHashJoin [codegen id : 10] +(22) BroadcastHashJoin [codegen id : 8] Left keys [1]: [item_sk#10] Right keys [1]: [item_sk#16] Join type: Inner Join condition: (rk#12 >= rk#15) -(20) Project [codegen id : 10] +(23) Project [codegen id : 8] Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] -(21) HashAggregate [codegen id : 10] +(24) HashAggregate [codegen id : 8] Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [partial_sum(sumws#17)] Aggregate Attributes [2]: [sum#18, isEmpty#19] Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(22) Exchange +(25) Exchange Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(23) HashAggregate [codegen id : 11] +(26) HashAggregate [codegen id : 9] Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [sum(sumws#17)] Aggregate Attributes [1]: [sum(sumws#17)#22] Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] -(24) Exchange +(27) Exchange Input [3]: [item_sk#10, d_date#6, cume_sales#23] Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(25) Sort [codegen id : 12] +(28) Sort [codegen id : 10] Input [3]: [item_sk#10, d_date#6, cume_sales#23] Arguments: [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(26) Scan parquet spark_catalog.default.store_sales +(29) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] @@ -193,224 +209,221 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(27) CometFilter +(30) CometFilter Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#24) -(28) ColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] - -(29) ReusedExchange [Reuses operator id: 72] +(31) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#28, d_date#29] -(30) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#28] -Join type: Inner -Join condition: None +(32) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Right output [2]: [d_date_sk#28, d_date#29] +Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight -(31) Project [codegen id : 14] -Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +(33) CometProject Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] +Arguments: [ss_item_sk#24, ss_sales_price#25, d_date#29], [ss_item_sk#24, ss_sales_price#25, d_date#29] -(32) HashAggregate [codegen id : 14] +(34) CometHashAggregate Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] Keys [2]: [ss_item_sk#24, d_date#29] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#30] -Results [3]: [ss_item_sk#24, d_date#29, sum#31] -(33) Exchange -Input [3]: [ss_item_sk#24, d_date#29, sum#31] +(35) ColumnarToRow [codegen id : 11] +Input [3]: [ss_item_sk#24, d_date#29, sum#30] + +(36) Exchange +Input [3]: [ss_item_sk#24, d_date#29, sum#30] Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(34) HashAggregate [codegen id : 15] -Input [3]: [ss_item_sk#24, d_date#29, sum#31] +(37) HashAggregate [codegen id : 12] +Input [3]: [ss_item_sk#24, d_date#29, sum#30] Keys [2]: [ss_item_sk#24, d_date#29] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#32] -Results [4]: [ss_item_sk#24 AS item_sk#33, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#32,17,2) AS sumss#34, ss_item_sk#24] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] +Results [4]: [ss_item_sk#24 AS item_sk#32, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS sumss#33, ss_item_sk#24] -(35) Exchange -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +(38) Exchange +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(36) Sort [codegen id : 16] -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +(39) Sort [codegen id : 13] +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] Arguments: [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 -(37) Window -Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] +(40) Window +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] -(38) Project [codegen id : 22] -Output [4]: [item_sk#33, d_date#29, sumss#34, rk#35] -Input [5]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24, rk#35] +(41) Project [codegen id : 18] +Output [4]: [item_sk#32, d_date#29, sumss#33, rk#34] +Input [5]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24, rk#34] -(39) ReusedExchange [Reuses operator id: 35] -Output [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] +(42) ReusedExchange [Reuses operator id: 38] +Output [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] -(40) Sort [codegen id : 20] -Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -Arguments: [ss_item_sk#37 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST], false, 0 +(43) Sort [codegen id : 16] +Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] +Arguments: [ss_item_sk#36 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 -(41) Window -Input [4]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37] -Arguments: [row_number() windowspecdefinition(ss_item_sk#37, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [ss_item_sk#37], [d_date#36 ASC NULLS FIRST] +(44) Window +Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] +Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#35 ASC NULLS FIRST] -(42) Project [codegen id : 21] -Output [3]: [item_sk#33 AS item_sk#39, sumss#34 AS sumss#40, rk#38] -Input [5]: [item_sk#33, d_date#36, sumss#34, ss_item_sk#37, rk#38] +(45) Project [codegen id : 17] +Output [3]: [item_sk#32 AS item_sk#38, sumss#33 AS sumss#39, rk#37] +Input [5]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36, rk#37] -(43) BroadcastExchange -Input [3]: [item_sk#39, sumss#40, rk#38] +(46) BroadcastExchange +Input [3]: [item_sk#38, sumss#39, rk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(44) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [item_sk#33] -Right keys [1]: [item_sk#39] +(47) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_sk#32] +Right keys [1]: [item_sk#38] Join type: Inner -Join condition: (rk#35 >= rk#38) - -(45) Project [codegen id : 22] -Output [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] -Input [7]: [item_sk#33, d_date#29, sumss#34, rk#35, item_sk#39, sumss#40, rk#38] - -(46) HashAggregate [codegen id : 22] -Input [4]: [item_sk#33, d_date#29, sumss#34, sumss#40] -Keys [3]: [item_sk#33, d_date#29, sumss#34] -Functions [1]: [partial_sum(sumss#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] - -(47) Exchange -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -Arguments: hashpartitioning(item_sk#33, d_date#29, sumss#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(48) HashAggregate [codegen id : 23] -Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] -Keys [3]: [item_sk#33, d_date#29, sumss#34] -Functions [1]: [sum(sumss#40)] -Aggregate Attributes [1]: [sum(sumss#40)#45] -Results [3]: [item_sk#33, d_date#29, sum(sumss#40)#45 AS cume_sales#46] - -(49) Exchange -Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: hashpartitioning(item_sk#33, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(50) Sort [codegen id : 24] -Input [3]: [item_sk#33, d_date#29, cume_sales#46] -Arguments: [item_sk#33 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 - -(51) SortMergeJoin [codegen id : 25] +Join condition: (rk#34 >= rk#37) + +(48) Project [codegen id : 18] +Output [4]: [item_sk#32, d_date#29, sumss#33, sumss#39] +Input [7]: [item_sk#32, d_date#29, sumss#33, rk#34, item_sk#38, sumss#39, rk#37] + +(49) HashAggregate [codegen id : 18] +Input [4]: [item_sk#32, d_date#29, sumss#33, sumss#39] +Keys [3]: [item_sk#32, d_date#29, sumss#33] +Functions [1]: [partial_sum(sumss#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] + +(50) Exchange +Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +Arguments: hashpartitioning(item_sk#32, d_date#29, sumss#33, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(51) HashAggregate [codegen id : 19] +Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +Keys [3]: [item_sk#32, d_date#29, sumss#33] +Functions [1]: [sum(sumss#39)] +Aggregate Attributes [1]: [sum(sumss#39)#44] +Results [3]: [item_sk#32, d_date#29, sum(sumss#39)#44 AS cume_sales#45] + +(52) Exchange +Input [3]: [item_sk#32, d_date#29, cume_sales#45] +Arguments: hashpartitioning(item_sk#32, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(53) Sort [codegen id : 20] +Input [3]: [item_sk#32, d_date#29, cume_sales#45] +Arguments: [item_sk#32 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 + +(54) SortMergeJoin [codegen id : 21] Left keys [2]: [item_sk#10, d_date#6] -Right keys [2]: [item_sk#33, d_date#29] +Right keys [2]: [item_sk#32, d_date#29] Join type: FullOuter Join condition: None -(52) Filter [codegen id : 25] -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END) +(55) Filter [codegen id : 21] +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END) -(53) Project [codegen id : 25] -Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END AS item_sk#47, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#46 AS store_sales#50] -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] +(56) Project [codegen id : 21] +Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END AS item_sk#46, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#47, cume_sales#23 AS web_sales#48, cume_sales#45 AS store_sales#49] +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] -(54) Exchange -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: hashpartitioning(item_sk#47, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(57) Exchange +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, [plan_id=11] -(55) Sort [codegen id : 26] -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], false, 0 +(58) Sort [codegen id : 22] +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], false, 0 -(56) Window -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#47], [d_date#48 ASC NULLS FIRST] +(59) Window +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 ASC NULLS FIRST] -(57) ReusedExchange [Reuses operator id: 54] -Output [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +(60) ReusedExchange [Reuses operator id: 57] +Output [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -(58) Sort [codegen id : 52] -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], false, 0 +(61) Sort [codegen id : 44] +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], false, 0 -(59) Window -Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#52], [item_sk#47], [d_date#48 ASC NULLS FIRST] +(62) Window +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#46], [d_date#47 ASC NULLS FIRST] -(60) Project [codegen id : 53] -Output [4]: [item_sk#47 AS item_sk#53, web_sales#49 AS web_sales#54, store_sales#50 AS store_sales#55, rk#52] -Input [5]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#52] +(63) Project [codegen id : 45] +Output [4]: [item_sk#46 AS item_sk#52, web_sales#48 AS web_sales#53, store_sales#49 AS store_sales#54, rk#51] +Input [5]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#51] -(61) BroadcastExchange -Input [4]: [item_sk#53, web_sales#54, store_sales#55, rk#52] +(64) BroadcastExchange +Input [4]: [item_sk#52, web_sales#53, store_sales#54, rk#51] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(62) BroadcastHashJoin [codegen id : 54] -Left keys [1]: [item_sk#47] -Right keys [1]: [item_sk#53] +(65) BroadcastHashJoin [codegen id : 46] +Left keys [1]: [item_sk#46] +Right keys [1]: [item_sk#52] Join type: Inner -Join condition: (rk#51 >= rk#52) - -(63) Project [codegen id : 54] -Output [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] -Input [9]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#51, item_sk#53, web_sales#54, store_sales#55, rk#52] - -(64) HashAggregate [codegen id : 54] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] -Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Functions [2]: [partial_max(web_sales#54), partial_max(store_sales#55)] -Aggregate Attributes [2]: [max#56, max#57] -Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] - -(65) HashAggregate [codegen id : 54] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#58, max#59] -Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] -Functions [2]: [max(web_sales#54), max(store_sales#55)] -Aggregate Attributes [2]: [max(web_sales#54)#60, max(store_sales#55)#61] -Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max(web_sales#54)#60 AS web_cumulative#62, max(store_sales#55)#61 AS store_cumulative#63] - -(66) Filter [codegen id : 54] -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] -Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) - -(67) TakeOrderedAndProject -Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] -Arguments: 100, [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#62, store_cumulative#63] +Join condition: (rk#50 >= rk#51) + +(66) Project [codegen id : 46] +Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] +Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#52, web_sales#53, store_sales#54, rk#51] + +(67) HashAggregate [codegen id : 46] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] +Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] +Aggregate Attributes [2]: [max#55, max#56] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] + +(68) HashAggregate [codegen id : 46] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] +Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Functions [2]: [max(web_sales#53), max(store_sales#54)] +Aggregate Attributes [2]: [max(web_sales#53)#59, max(store_sales#54)#60] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#59 AS web_cumulative#61, max(store_sales#54)#60 AS store_cumulative#62] + +(69) Filter [codegen id : 46] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] +Condition : ((isnotnull(web_cumulative#61) AND isnotnull(store_cumulative#62)) AND (web_cumulative#61 > store_cumulative#62)) + +(70) TakeOrderedAndProject +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] +Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) -(68) Scan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +(71) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] -Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) +(72) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] +Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(70) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] +(73) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(71) ColumnarToRow [codegen id : 1] +(74) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(72) BroadcastExchange +(75) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index 3109290dc7..78dd29c2ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (54) + WholeStageCodegen (46) Filter [web_cumulative,store_cumulative] HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] @@ -7,44 +7,44 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (26) + WholeStageCodegen (22) Sort [item_sk,d_date] InputAdapter Exchange [item_sk] #1 - WholeStageCodegen (25) + WholeStageCodegen (21) Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] Filter [item_sk,item_sk] SortMergeJoin [item_sk,d_date,item_sk,d_date] InputAdapter - WholeStageCodegen (12) + WholeStageCodegen (10) Sort [item_sk,d_date] InputAdapter Exchange [item_sk,d_date] #2 - WholeStageCodegen (11) + WholeStageCodegen (9) HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] InputAdapter Exchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) + WholeStageCodegen (8) HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] Project [item_sk,d_date,sumws,sumws] BroadcastHashJoin [item_sk,item_sk,rk,rk] Project [item_sk,d_date,sumws,rk] InputAdapter Window [ws_item_sk,d_date] - WholeStageCodegen (4) + WholeStageCodegen (3) Sort [ws_item_sk,d_date] InputAdapter Exchange [ws_item_sk] #4 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] InputAdapter Exchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [ws_item_sk,d_date,ws_sales_price] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -55,70 +55,71 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometProject [d_date_sk,d_date] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + CometBroadcastExchange #7 + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) + BroadcastExchange #8 + WholeStageCodegen (7) Project [item_sk,sumws,rk] InputAdapter Window [ws_item_sk,d_date] - WholeStageCodegen (8) + WholeStageCodegen (6) Sort [ws_item_sk,d_date] InputAdapter ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 InputAdapter - WholeStageCodegen (24) + WholeStageCodegen (20) Sort [item_sk,d_date] InputAdapter - Exchange [item_sk,d_date] #8 - WholeStageCodegen (23) + Exchange [item_sk,d_date] #9 + WholeStageCodegen (19) HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] InputAdapter - Exchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (22) + Exchange [item_sk,d_date,sumss] #10 + WholeStageCodegen (18) HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] Project [item_sk,d_date,sumss,sumss] BroadcastHashJoin [item_sk,item_sk,rk,rk] Project [item_sk,d_date,sumss,rk] InputAdapter Window [ss_item_sk,d_date] - WholeStageCodegen (16) + WholeStageCodegen (13) Sort [ss_item_sk,d_date] InputAdapter - Exchange [ss_item_sk] #10 - WholeStageCodegen (15) + Exchange [ss_item_sk] #11 + WholeStageCodegen (12) HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] InputAdapter - Exchange [ss_item_sk,d_date] #11 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [ss_item_sk,d_date] #12 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometHashAggregate [ss_item_sk,d_date,ss_sales_price] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + ReusedExchange [d_date_sk,d_date] #7 InputAdapter - BroadcastExchange #12 - WholeStageCodegen (21) + BroadcastExchange #13 + WholeStageCodegen (17) Project [item_sk,sumss,rk] InputAdapter Window [ss_item_sk,d_date] - WholeStageCodegen (20) + WholeStageCodegen (16) Sort [ss_item_sk,d_date] InputAdapter - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (53) + BroadcastExchange #14 + WholeStageCodegen (45) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (52) + WholeStageCodegen (44) Sort [item_sk,d_date] InputAdapter ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index dbcade8ca6..1b66eb4da0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -13,23 +13,23 @@ TakeOrderedAndProject (45) : : +- Exchange (22) : : +- * HashAggregate (21) : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) + : : +- * ColumnarToRow (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (15) : : +- CometFilter (14) : : +- CometScan parquet spark_catalog.default.call_center (13) : +- BroadcastExchange (35) @@ -57,10 +57,7 @@ ReadSchema: struct Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) -(3) ColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] - -(4) Scan parquet spark_catalog.default.catalog_sales +(3) Scan parquet spark_catalog.default.catalog_sales Output [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -68,39 +65,46 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#7), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] Condition : (isnotnull(cs_item_sk#5) AND isnotnull(cs_call_center_sk#4)) -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] - -(7) BroadcastExchange +(5) CometBroadcastExchange Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +Arguments: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#5] -Join type: Inner -Join condition: None +(6) CometBroadcastHashJoin +Left output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Right output [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Arguments: [i_item_sk#1], [cs_item_sk#5], Inner, BuildRight -(9) Project [codegen id : 4] -Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] +(7) CometProject Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Arguments: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7], [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 49] +(8) Scan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [d_date_sk#9, d_year#10, d_moy#11] + +(11) CometBroadcastHashJoin +Left output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] +Right output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: [cs_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(12) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] +(12) CometProject Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] +Arguments: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11], [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] (13) Scan parquet spark_catalog.default.call_center Output [2]: [cc_call_center_sk#12, cc_name#13] @@ -113,145 +117,142 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#12, cc_name#13] Condition : (isnotnull(cc_call_center_sk#12) AND isnotnull(cc_name#13)) -(15) ColumnarToRow [codegen id : 3] +(15) CometBroadcastExchange Input [2]: [cc_call_center_sk#12, cc_name#13] +Arguments: [cc_call_center_sk#12, cc_name#13] -(16) BroadcastExchange -Input [2]: [cc_call_center_sk#12, cc_name#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(16) CometBroadcastHashJoin +Left output [6]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] +Right output [2]: [cc_call_center_sk#12, cc_name#13] +Arguments: [cs_call_center_sk#4], [cc_call_center_sk#12], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#4] -Right keys [1]: [cc_call_center_sk#12] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] +(17) CometProject Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11, cc_call_center_sk#12, cc_name#13] +Arguments: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13], [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] -(19) HashAggregate [codegen id : 4] +(18) CometHashAggregate Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [1]: [sum#14] -Results [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] + +(19) ColumnarToRow [codegen id : 1] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#14] (20) Exchange -Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#14] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] +(21) HashAggregate [codegen id : 2] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#14] Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#16] -Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS _w0#18] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#15] +Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#15,17,2) AS sum_sales#16, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#15,17,2) AS _w0#17] (22) Exchange -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(23) Sort [codegen id : 6] -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +(23) Sort [codegen id : 3] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 (24) Window -Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#18], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] -(25) Filter [codegen id : 7] -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +(25) Filter [codegen id : 4] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) (26) Window -Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18] +Arguments: [avg(_w0#17) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#19], [i_category#3, i_brand#2, cc_name#13, d_year#10] -(27) Filter [codegen id : 22] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) +(27) Filter [codegen id : 13] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] +Condition : ((isnotnull(avg_monthly_sales#19) AND (avg_monthly_sales#19 > 0.000000)) AND CASE WHEN (avg_monthly_sales#19 > 0.000000) THEN ((abs((sum_sales#16 - avg_monthly_sales#19)) / avg_monthly_sales#19) > 0.1000000000000000) END) -(28) Project [codegen id : 22] -Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +(28) Project [codegen id : 13] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, _w0#17, rn#18, avg_monthly_sales#19] (29) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] +Output [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] -(30) HashAggregate [codegen id : 12] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] -Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#17] +(30) HashAggregate [codegen id : 6] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum#25] +Keys [5]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24] +Functions [1]: [sum(UnscaledValue(cs_sales_price#26))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#26))#15] +Results [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, MakeDecimal(sum(UnscaledValue(cs_sales_price#26))#15,17,2) AS sum_sales#16] (31) Exchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: hashpartitioning(i_category#20, i_brand#21, cc_name#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(32) Sort [codegen id : 13] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 +(32) Sort [codegen id : 7] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [i_category#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, cc_name#22 ASC NULLS FIRST, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST], false, 0 (33) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Input [6]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16] +Arguments: [rank(d_year#23, d_moy#24) windowspecdefinition(i_category#20, i_brand#21, cc_name#22, d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#27], [i_category#20, i_brand#21, cc_name#22], [d_year#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST] -(34) Project [codegen id : 14] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#17 AS sum_sales#29, rn#28] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#17, rn#28] +(34) Project [codegen id : 8] +Output [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#16 AS sum_sales#28, rn#27] +Input [7]: [i_category#20, i_brand#21, cc_name#22, d_year#23, d_moy#24, sum_sales#16, rn#27] (35) BroadcastExchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] +Input [5]: [i_category#20, i_brand#21, cc_name#22, sum_sales#28, rn#27] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#28 + 1)] +(36) BroadcastHashJoin [codegen id : 13] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] +Right keys [4]: [i_category#20, i_brand#21, cc_name#22, (rn#27 + 1)] Join type: Inner Join condition: None -(37) Project [codegen id : 22] -Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#29, rn#28] +(37) Project [codegen id : 13] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#28] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, i_category#20, i_brand#21, cc_name#22, sum_sales#28, rn#27] (38) ReusedExchange [Reuses operator id: 31] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] +Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] -(39) Sort [codegen id : 20] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 +(39) Sort [codegen id : 11] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (40) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#34], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -(41) Project [codegen id : 21] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#17 AS sum_sales#36, rn#35] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#17, rn#35] +(41) Project [codegen id : 12] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#16 AS sum_sales#35, rn#34] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#16, rn#34] (42) BroadcastExchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#35, rn#34] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(43) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#35 - 1)] +(43) BroadcastHashJoin [codegen id : 13] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#18] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#34 - 1)] Join type: Inner Join condition: None -(44) Project [codegen id : 22] -Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#36, rn#35] +(44) Project [codegen id : 13] +Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, sum_sales#28 AS psum#36, sum_sales#35 AS nsum#37] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#16, avg_monthly_sales#19, rn#18, sum_sales#28, i_category#29, i_brand#30, cc_name#31, sum_sales#35, rn#34] (45) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#36, nsum#37] +Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#19, sum_sales#16, psum#36, nsum#37] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (49) +- * ColumnarToRow (48) +- CometFilter (47) @@ -274,6 +275,6 @@ Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (49) BroadcastExchange Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index 56e33be9e4..c2c8e089f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,74 +8,67 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter Exchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] InputAdapter Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,cs_item_sk] CometFilter [i_item_sk,i_category,i_brand] CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [cs_item_sk,cs_call_center_sk] - CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_moy,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange #3 + CometFilter [cs_item_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #5 + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) + BroadcastExchange #7 + WholeStageCodegen (8) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) + WholeStageCodegen (7) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - Exchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) + Exchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (6) HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] InputAdapter ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) + BroadcastExchange #9 + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) + WholeStageCodegen (11) Sort [i_category,i_brand,cc_name,d_year,d_moy] InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 2b45472ff4..3ca1d35517 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -8,68 +8,68 @@ TakeOrderedAndProject (83) : +- Exchange (67) : +- * HashAggregate (66) : +- Union (65) - : :- * HashAggregate (20) - : : +- Exchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (11) - : : : +- * BroadcastHashJoin Inner BuildRight (10) - : : : :- * ColumnarToRow (8) - : : : : +- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : +- ReusedExchange (9) - : : +- BroadcastExchange (15) - : : +- * ColumnarToRow (14) - : : +- CometFilter (13) - : : +- CometScan parquet spark_catalog.default.store (12) - : :- * HashAggregate (40) - : : +- Exchange (39) - : : +- * HashAggregate (38) - : : +- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * ColumnarToRow (28) - : : : : +- CometUnion (27) - : : : : :- CometProject (23) - : : : : : +- CometFilter (22) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) - : : : : +- CometProject (26) - : : : : +- CometFilter (25) - : : : : +- CometScan parquet spark_catalog.default.catalog_returns (24) - : : : +- ReusedExchange (29) - : : +- BroadcastExchange (35) - : : +- * ColumnarToRow (34) - : : +- CometFilter (33) - : : +- CometScan parquet spark_catalog.default.catalog_page (32) + : :- * HashAggregate (22) + : : +- Exchange (21) + : : +- * ColumnarToRow (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometProject (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.store (14) + : :- * HashAggregate (41) + : : +- Exchange (40) + : : +- * ColumnarToRow (39) + : : +- CometHashAggregate (38) + : : +- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (30) + : : +- CometBroadcastExchange (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.catalog_page (33) : +- * HashAggregate (64) : +- Exchange (63) - : +- * HashAggregate (62) - : +- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Project (55) - : : +- * BroadcastHashJoin Inner BuildRight (54) - : : :- * ColumnarToRow (52) - : : : +- CometUnion (51) - : : : :- CometProject (43) - : : : : +- CometFilter (42) - : : : : +- CometScan parquet spark_catalog.default.web_sales (41) - : : : +- CometProject (50) - : : : +- CometBroadcastHashJoin (49) - : : : :- CometBroadcastExchange (45) - : : : : +- CometScan parquet spark_catalog.default.web_returns (44) - : : : +- CometProject (48) - : : : +- CometFilter (47) - : : : +- CometScan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) + : +- * ColumnarToRow (62) + : +- CometHashAggregate (61) + : +- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometUnion (52) + : : : :- CometProject (44) + : : : : +- CometFilter (43) + : : : : +- CometScan parquet spark_catalog.default.web_sales (42) + : : : +- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometBroadcastExchange (46) + : : : : +- CometScan parquet spark_catalog.default.web_returns (45) + : : : +- CometProject (49) + : : : +- CometFilter (48) + : : : +- CometScan parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (53) + : +- CometBroadcastExchange (58) : +- CometFilter (57) : +- CometScan parquet spark_catalog.default.web_site (56) :- * HashAggregate (73) @@ -120,382 +120,377 @@ Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, n Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] -(8) ColumnarToRow [codegen id : 3] -Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] +ReadSchema: struct + +(9) CometFilter +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(9) ReusedExchange [Reuses operator id: 88] -Output [1]: [d_date_sk#22] +(10) CometProject +Input [2]: [d_date_sk#22, d_date#23] +Arguments: [d_date_sk#22], [d_date_sk#22] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: [d_date_sk#22] -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [1]: [d_date_sk#22] +Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight -(11) Project [codegen id : 3] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +(13) CometProject Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] +Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -(12) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] +(14) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_store_id#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(13) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) +(15) CometFilter +Input [2]: [s_store_sk#24, s_store_id#25] +Condition : isnotnull(s_store_sk#24) -(14) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#23, s_store_id#24] +(16) CometBroadcastExchange +Input [2]: [s_store_sk#24, s_store_id#25] +Arguments: [s_store_sk#24, s_store_id#25] -(15) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(17) CometBroadcastHashJoin +Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] +Right output [2]: [s_store_sk#24, s_store_id#25] +Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight -(16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None +(18) CometProject +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] +Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -(17) Project [codegen id : 3] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#24] - -(18) HashAggregate [codegen id : 3] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] -Keys [1]: [s_store_id#24] +(19) CometHashAggregate +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] +Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] -Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -(19) Exchange -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(20) ColumnarToRow [codegen id : 1] +Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] + +(21) Exchange +Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(20) HashAggregate [codegen id : 4] -Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] -Keys [1]: [s_store_id#24] +(22) HashAggregate [codegen id : 2] +Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] +Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] -Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#30, sum(UnscaledValue(return_amt#10))#31, sum(UnscaledValue(profit#9))#32, sum(UnscaledValue(net_loss#11))#33] +Results [5]: [store channel AS channel#34, concat(store, s_store_id#25) AS id#35, MakeDecimal(sum(UnscaledValue(sales_price#8))#30,17,2) AS sales#36, MakeDecimal(sum(UnscaledValue(return_amt#10))#31,17,2) AS returns#37, (MakeDecimal(sum(UnscaledValue(profit#9))#32,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#33,17,2)) AS profit#38] -(21) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +(23) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(22) CometFilter -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Condition : isnotnull(cs_catalog_page_sk#42) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +Condition : isnotnull(cs_catalog_page_sk#39) -(23) CometProject -Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] -Arguments: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52], [cs_catalog_page_sk#42 AS page_sk#47, cs_sold_date_sk#45 AS date_sk#48, cs_ext_sales_price#43 AS sales_price#49, cs_net_profit#44 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] +(25) CometProject +Input [4]: [cs_catalog_page_sk#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +Arguments: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49], [cs_catalog_page_sk#39 AS page_sk#44, cs_sold_date_sk#42 AS date_sk#45, cs_ext_sales_price#40 AS sales_price#46, cs_net_profit#41 AS profit#47, 0.00 AS return_amt#48, 0.00 AS net_loss#49] -(24) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +(26) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#50, cr_return_amount#51, cr_net_loss#52, cr_returned_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(cr_returned_date_sk#53), dynamicpruningexpression(cr_returned_date_sk#53 IN dynamicpruning#43)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Condition : isnotnull(cr_catalog_page_sk#53) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#50, cr_return_amount#51, cr_net_loss#52, cr_returned_date_sk#53] +Condition : isnotnull(cr_catalog_page_sk#50) -(26) CometProject -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Arguments: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62], [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] +(28) CometProject +Input [4]: [cr_catalog_page_sk#50, cr_return_amount#51, cr_net_loss#52, cr_returned_date_sk#53] +Arguments: [page_sk#54, date_sk#55, sales_price#56, profit#57, return_amt#58, net_loss#59], [cr_catalog_page_sk#50 AS page_sk#54, cr_returned_date_sk#53 AS date_sk#55, 0.00 AS sales_price#56, 0.00 AS profit#57, cr_return_amount#51 AS return_amt#58, cr_net_loss#52 AS net_loss#59] -(27) CometUnion -Child 0 Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] -Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62] +(29) CometUnion +Child 0 Input [6]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49] +Child 1 Input [6]: [page_sk#54, date_sk#55, sales_price#56, profit#57, return_amt#58, net_loss#59] -(28) ColumnarToRow [codegen id : 7] -Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#60] -(29) ReusedExchange [Reuses operator id: 88] -Output [1]: [d_date_sk#63] +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49] +Right output [1]: [d_date_sk#60] +Arguments: [date_sk#45], [d_date_sk#60], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [date_sk#48] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None +(32) CometProject +Input [7]: [page_sk#44, date_sk#45, sales_price#46, profit#47, return_amt#48, net_loss#49, d_date_sk#60] +Arguments: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49], [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49] -(31) Project [codegen id : 7] -Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] -Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] - -(32) Scan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +(33) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(33) CometFilter -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) - -(34) ColumnarToRow [codegen id : 6] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] - -(35) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(36) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 7] -Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] -Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#65] - -(38) HashAggregate [codegen id : 7] -Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] -Keys [1]: [cp_catalog_page_id#65] -Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum#66, sum#67, sum#68, sum#69] -Results [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] - -(39) Exchange -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(40) HashAggregate [codegen id : 8] -Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] -Keys [1]: [cp_catalog_page_id#65] -Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] -Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#65) AS id#79, MakeDecimal(sum(UnscaledValue(sales_price#49))#74,17,2) AS sales#80, MakeDecimal(sum(UnscaledValue(return_amt#51))#75,17,2) AS returns#81, (MakeDecimal(sum(UnscaledValue(profit#50))#76,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#77,17,2)) AS profit#82] - -(41) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +(34) CometFilter +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) + +(35) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [cp_catalog_page_sk#61, cp_catalog_page_id#62] + +(36) CometBroadcastHashJoin +Left output [5]: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49] +Right output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [page_sk#44], [cp_catalog_page_sk#61], Inner, BuildRight + +(37) CometProject +Input [7]: [page_sk#44, sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62], [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62] + +(38) CometHashAggregate +Input [5]: [sales_price#46, profit#47, return_amt#48, net_loss#49, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] +Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(UnscaledValue(return_amt#48)), partial_sum(UnscaledValue(profit#47)), partial_sum(UnscaledValue(net_loss#49))] + +(39) ColumnarToRow [codegen id : 3] +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] + +(40) Exchange +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(41) HashAggregate [codegen id : 4] +Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] +Keys [1]: [cp_catalog_page_id#62] +Functions [4]: [sum(UnscaledValue(sales_price#46)), sum(UnscaledValue(return_amt#48)), sum(UnscaledValue(profit#47)), sum(UnscaledValue(net_loss#49))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#46))#67, sum(UnscaledValue(return_amt#48))#68, sum(UnscaledValue(profit#47))#69, sum(UnscaledValue(net_loss#49))#70] +Results [5]: [catalog channel AS channel#71, concat(catalog_page, cp_catalog_page_id#62) AS id#72, MakeDecimal(sum(UnscaledValue(sales_price#46))#67,17,2) AS sales#73, MakeDecimal(sum(UnscaledValue(return_amt#48))#68,17,2) AS returns#74, (MakeDecimal(sum(UnscaledValue(profit#47))#69,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#49))#70,17,2)) AS profit#75] + +(42) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#80)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(42) CometFilter -Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] -Condition : isnotnull(ws_web_site_sk#83) +(43) CometFilter +Input [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_web_site_sk#76) -(43) CometProject -Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] -Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] +(44) CometProject +Input [4]: [ws_web_site_sk#76, ws_ext_sales_price#77, ws_net_profit#78, ws_sold_date_sk#79] +Arguments: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86], [ws_web_site_sk#76 AS wsr_web_site_sk#81, ws_sold_date_sk#79 AS date_sk#82, ws_ext_sales_price#77 AS sales_price#83, ws_net_profit#78 AS profit#84, 0.00 AS return_amt#85, 0.00 AS net_loss#86] -(44) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +(45) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#80)] ReadSchema: struct -(45) CometBroadcastExchange -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Arguments: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -(46) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +(47) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(47) CometFilter -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) - -(48) CometProject -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +(48) CometFilter +Input [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] +Condition : ((isnotnull(ws_item_sk#92) AND isnotnull(ws_order_number#94)) AND isnotnull(ws_web_site_sk#93)) -(49) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner +(49) CometProject +Input [4]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94, ws_sold_date_sk#95] +Arguments: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94], [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] -(50) CometProject -Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Right output [3]: [ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] +Arguments: [wr_item_sk#87, wr_order_number#88], [ws_item_sk#92, ws_order_number#94], Inner, BuildLeft -(51) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] -Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] +(51) CometProject +Input [8]: [wr_item_sk#87, wr_order_number#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, ws_item_sk#92, ws_web_site_sk#93, ws_order_number#94] +Arguments: [wsr_web_site_sk#96, date_sk#97, sales_price#98, profit#99, return_amt#100, net_loss#101], [ws_web_site_sk#93 AS wsr_web_site_sk#96, wr_returned_date_sk#91 AS date_sk#97, 0.00 AS sales_price#98, 0.00 AS profit#99, wr_return_amt#89 AS return_amt#100, wr_net_loss#90 AS net_loss#101] -(52) ColumnarToRow [codegen id : 11] -Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86] +Child 1 Input [6]: [wsr_web_site_sk#96, date_sk#97, sales_price#98, profit#99, return_amt#100, net_loss#101] -(53) ReusedExchange [Reuses operator id: 88] -Output [1]: [d_date_sk#109] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#102] -(54) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#89] -Right keys [1]: [d_date_sk#109] -Join type: Inner -Join condition: None +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86] +Right output [1]: [d_date_sk#102] +Arguments: [date_sk#82], [d_date_sk#102], Inner, BuildRight -(55) Project [codegen id : 11] -Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] -Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] +(55) CometProject +Input [7]: [wsr_web_site_sk#81, date_sk#82, sales_price#83, profit#84, return_amt#85, net_loss#86, d_date_sk#102] +Arguments: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86], [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86] (56) Scan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#110, web_site_id#111] +Output [2]: [web_site_sk#103, web_site_id#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (57) CometFilter -Input [2]: [web_site_sk#110, web_site_id#111] -Condition : isnotnull(web_site_sk#110) +Input [2]: [web_site_sk#103, web_site_id#104] +Condition : isnotnull(web_site_sk#103) -(58) ColumnarToRow [codegen id : 10] -Input [2]: [web_site_sk#110, web_site_id#111] +(58) CometBroadcastExchange +Input [2]: [web_site_sk#103, web_site_id#104] +Arguments: [web_site_sk#103, web_site_id#104] -(59) BroadcastExchange -Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] +(59) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86] +Right output [2]: [web_site_sk#103, web_site_id#104] +Arguments: [wsr_web_site_sk#81], [web_site_sk#103], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [wsr_web_site_sk#88] -Right keys [1]: [web_site_sk#110] -Join type: Inner -Join condition: None +(60) CometProject +Input [7]: [wsr_web_site_sk#81, sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_sk#103, web_site_id#104] +Arguments: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104], [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104] -(61) Project [codegen id : 11] -Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] -Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] +(61) CometHashAggregate +Input [5]: [sales_price#83, profit#84, return_amt#85, net_loss#86, web_site_id#104] +Keys [1]: [web_site_id#104] +Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(UnscaledValue(return_amt#85)), partial_sum(UnscaledValue(profit#84)), partial_sum(UnscaledValue(net_loss#86))] -(62) HashAggregate [codegen id : 11] -Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] -Keys [1]: [web_site_id#111] -Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] -Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] +(62) ColumnarToRow [codegen id : 5] +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] (63) Exchange -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(64) HashAggregate [codegen id : 12] -Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] -Keys [1]: [web_site_id#111] -Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] -Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] +(64) HashAggregate [codegen id : 6] +Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] +Keys [1]: [web_site_id#104] +Functions [4]: [sum(UnscaledValue(sales_price#83)), sum(UnscaledValue(return_amt#85)), sum(UnscaledValue(profit#84)), sum(UnscaledValue(net_loss#86))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#83))#109, sum(UnscaledValue(return_amt#85))#110, sum(UnscaledValue(profit#84))#111, sum(UnscaledValue(net_loss#86))#112] +Results [5]: [web channel AS channel#113, concat(web_site, web_site_id#104) AS id#114, MakeDecimal(sum(UnscaledValue(sales_price#83))#109,17,2) AS sales#115, MakeDecimal(sum(UnscaledValue(return_amt#85))#110,17,2) AS returns#116, (MakeDecimal(sum(UnscaledValue(profit#84))#111,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#86))#112,17,2)) AS profit#117] (65) Union -(66) HashAggregate [codegen id : 13] -Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] -Keys [2]: [channel#37, id#38] -Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] -Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +(66) HashAggregate [codegen id : 7] +Input [5]: [channel#34, id#35, sales#36, returns#37, profit#38] +Keys [2]: [channel#34, id#35] +Functions [3]: [partial_sum(sales#36), partial_sum(returns#37), partial_sum(profit#38)] +Aggregate Attributes [6]: [sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Results [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] (67) Exchange -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(68) HashAggregate [codegen id : 14] -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] -Results [5]: [channel#37, id#38, cast(sum(sales#39)#141 as decimal(37,2)) AS sales#144, cast(sum(returns#40)#142 as decimal(37,2)) AS returns#145, cast(sum(profit#41)#143 as decimal(38,2)) AS profit#146] +(68) HashAggregate [codegen id : 8] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Keys [2]: [channel#34, id#35] +Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#36)#130, sum(returns#37)#131, sum(profit#38)#132] +Results [5]: [channel#34, id#35, cast(sum(sales#36)#130 as decimal(37,2)) AS sales#133, cast(sum(returns#37)#131 as decimal(37,2)) AS returns#134, cast(sum(profit#38)#132 as decimal(38,2)) AS profit#135] (69) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] - -(70) HashAggregate [codegen id : 28] -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] -Results [4]: [channel#37, sum(sales#39)#141 AS sales#147, sum(returns#40)#142 AS returns#148, sum(profit#41)#143 AS profit#149] - -(71) HashAggregate [codegen id : 28] -Input [4]: [channel#37, sales#147, returns#148, profit#149] -Keys [1]: [channel#37] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -Results [7]: [channel#37, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(70) HashAggregate [codegen id : 16] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Keys [2]: [channel#34, id#35] +Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#36)#130, sum(returns#37)#131, sum(profit#38)#132] +Results [4]: [channel#34, sum(sales#36)#130 AS sales#136, sum(returns#37)#131 AS returns#137, sum(profit#38)#132 AS profit#138] + +(71) HashAggregate [codegen id : 16] +Input [4]: [channel#34, sales#136, returns#137, profit#138] +Keys [1]: [channel#34] +Functions [3]: [partial_sum(sales#136), partial_sum(returns#137), partial_sum(profit#138)] +Aggregate Attributes [6]: [sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Results [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] (72) Exchange -Input [7]: [channel#37, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] -Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] +Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(73) HashAggregate [codegen id : 29] -Input [7]: [channel#37, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] -Keys [1]: [channel#37] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#162, sum(returns#148)#163, sum(profit#149)#164] -Results [5]: [channel#37, null AS id#165, sum(sales#147)#162 AS sum(sales)#166, sum(returns#148)#163 AS sum(returns)#167, sum(profit#149)#164 AS sum(profit)#168] +(73) HashAggregate [codegen id : 17] +Input [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] +Keys [1]: [channel#34] +Functions [3]: [sum(sales#136), sum(returns#137), sum(profit#138)] +Aggregate Attributes [3]: [sum(sales#136)#151, sum(returns#137)#152, sum(profit#138)#153] +Results [5]: [channel#34, null AS id#154, sum(sales#136)#151 AS sum(sales)#155, sum(returns#137)#152 AS sum(returns)#156, sum(profit#138)#153 AS sum(profit)#157] (74) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(75) HashAggregate [codegen id : 43] -Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] -Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] -Results [3]: [sum(sales#39)#141 AS sales#147, sum(returns#40)#142 AS returns#148, sum(profit#41)#143 AS profit#149] +(75) HashAggregate [codegen id : 25] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Keys [2]: [channel#34, id#35] +Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#36)#130, sum(returns#37)#131, sum(profit#38)#132] +Results [3]: [sum(sales#36)#130 AS sales#136, sum(returns#37)#131 AS returns#137, sum(profit#38)#132 AS profit#138] -(76) HashAggregate [codegen id : 43] -Input [3]: [sales#147, returns#148, profit#149] +(76) HashAggregate [codegen id : 25] +Input [3]: [sales#136, returns#137, profit#138] Keys: [] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174] -Results [6]: [sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +Functions [3]: [partial_sum(sales#136), partial_sum(returns#137), partial_sum(profit#138)] +Aggregate Attributes [6]: [sum#158, isEmpty#159, sum#160, isEmpty#161, sum#162, isEmpty#163] +Results [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] (77) Exchange -Input [6]: [sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] +Input [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(78) HashAggregate [codegen id : 44] -Input [6]: [sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +(78) HashAggregate [codegen id : 26] +Input [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] Keys: [] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#181, sum(returns#148)#182, sum(profit#149)#183] -Results [5]: [null AS channel#184, null AS id#185, sum(sales#147)#181 AS sum(sales)#186, sum(returns#148)#182 AS sum(returns)#187, sum(profit#149)#183 AS sum(profit)#188] +Functions [3]: [sum(sales#136), sum(returns#137), sum(profit#138)] +Aggregate Attributes [3]: [sum(sales#136)#170, sum(returns#137)#171, sum(profit#138)#172] +Results [5]: [null AS channel#173, null AS id#174, sum(sales#136)#170 AS sum(sales)#175, sum(returns#137)#171 AS sum(returns)#176, sum(profit#138)#172 AS sum(profit)#177] (79) Union -(80) HashAggregate [codegen id : 45] -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +(80) HashAggregate [codegen id : 27] +Input [5]: [channel#34, id#35, sales#133, returns#134, profit#135] +Keys [5]: [channel#34, id#35, sales#133, returns#134, profit#135] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Results [5]: [channel#34, id#35, sales#133, returns#134, profit#135] (81) Exchange -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Arguments: hashpartitioning(channel#37, id#38, sales#144, returns#145, profit#146, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [5]: [channel#34, id#35, sales#133, returns#134, profit#135] +Arguments: hashpartitioning(channel#34, id#35, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(82) HashAggregate [codegen id : 46] -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +(82) HashAggregate [codegen id : 28] +Input [5]: [channel#34, id#35, sales#133, returns#134, profit#135] +Keys [5]: [channel#34, id#35, sales#133, returns#134, profit#135] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Results [5]: [channel#34, id#35, sales#133, returns#134, profit#135] (83) TakeOrderedAndProject -Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] -Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#144, returns#145, profit#146] +Input [5]: [channel#34, id#35, sales#133, returns#134, profit#135] +Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#133, returns#134, profit#135] ===== Subqueries ===== @@ -508,18 +503,18 @@ BroadcastExchange (88) (84) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#189] +Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct (85) CometFilter -Input [2]: [d_date_sk#22, d_date#189] -Condition : (((isnotnull(d_date#189) AND (d_date#189 >= 1998-08-04)) AND (d_date#189 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) (86) CometProject -Input [2]: [d_date_sk#22, d_date#189] +Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] (87) ColumnarToRow [codegen id : 1] @@ -527,16 +522,16 @@ Input [1]: [d_date_sk#22] (88) BroadcastExchange Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#53 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index aaec304fd0..dd66c9582f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -1,32 +1,32 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (46) + WholeStageCodegen (28) HashAggregate [channel,id,sales,returns,profit] InputAdapter Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (45) + WholeStageCodegen (27) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (14) + WholeStageCodegen (8) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #2 - WholeStageCodegen (13) + WholeStageCodegen (7) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter Exchange [s_store_id] #3 - WholeStageCodegen (3) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,s_store_sk] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [ss_store_sk] @@ -43,27 +43,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (8) + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (4) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter - Exchange [cp_catalog_page_id] #6 - WholeStageCodegen (7) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [cp_catalog_page_id] #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometHashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,cp_catalog_page_sk] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [cs_catalog_page_sk] @@ -73,27 +71,22 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [cr_catalog_page_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - ColumnarToRow - InputAdapter - CometFilter [cp_catalog_page_sk] - CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (12) + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #8 + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (6) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter - Exchange [web_site_id] #8 - WholeStageCodegen (11) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [web_site_id] #9 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,web_site_sk] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [date_sk,d_date_sk] CometUnion CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [ws_web_site_sk] @@ -101,35 +94,30 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - CometBroadcastExchange #9 + CometBroadcastExchange #10 CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (10) - ColumnarToRow - InputAdapter - CometFilter [web_site_sk] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (29) + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #11 + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (17) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [channel] #11 - WholeStageCodegen (28) + Exchange [channel] #12 + WholeStageCodegen (16) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (44) + WholeStageCodegen (26) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange #12 - WholeStageCodegen (43) + Exchange #13 + WholeStageCodegen (25) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index d7f619f205..fc156f98e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -1,43 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (39) -+- * Filter (38) - +- * HashAggregate (37) - +- Exchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.customer_address (1) - : : : +- BroadcastExchange (7) - : : : +- * ColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * ColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * ColumnarToRow (21) - : +- CometFilter (20) - : +- CometScan parquet spark_catalog.default.item (19) - +- BroadcastExchange (29) - +- * Filter (28) - +- * HashAggregate (27) - +- Exchange (26) - +- * ColumnarToRow (25) - +- CometHashAggregate (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.item (22) +TakeOrderedAndProject (40) ++- * Filter (39) + +- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometProject (7) + : : : +- CometBroadcastHashJoin (6) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer_address (1) + : : : +- CometBroadcastExchange (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.customer (3) + : : +- CometBroadcastExchange (10) + : : +- CometFilter (9) + : : +- CometScan parquet spark_catalog.default.store_sales (8) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.date_dim (13) + +- BroadcastExchange (33) + +- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- BroadcastExchange (30) + +- * Filter (29) + +- * HashAggregate (28) + +- Exchange (27) + +- * ColumnarToRow (26) + +- CometHashAggregate (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.item (23) (1) Scan parquet spark_catalog.default.customer_address @@ -51,38 +52,31 @@ ReadSchema: struct Input [2]: [ca_address_sk#1, ca_state#2] Condition : isnotnull(ca_address_sk#1) -(3) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#1, ca_state#2] - -(4) Scan parquet spark_catalog.default.customer +(3) Scan parquet spark_catalog.default.customer Output [2]: [c_customer_sk#3, c_current_addr_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [2]: [c_customer_sk#3, c_current_addr_sk#4] Condition : (isnotnull(c_current_addr_sk#4) AND isnotnull(c_customer_sk#3)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [2]: [c_customer_sk#3, c_current_addr_sk#4] +Arguments: [c_customer_sk#3, c_current_addr_sk#4] -(7) BroadcastExchange -Input [2]: [c_customer_sk#3, c_current_addr_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [2]: [ca_address_sk#1, ca_state#2] +Right output [2]: [c_customer_sk#3, c_current_addr_sk#4] +Arguments: [ca_address_sk#1], [c_current_addr_sk#4], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ca_address_sk#1] -Right keys [1]: [c_current_addr_sk#4] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 7] -Output [2]: [ca_state#2, c_customer_sk#3] +(7) CometProject Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4] +Arguments: [ca_state#2, c_customer_sk#3], [ca_state#2, c_customer_sk#3] -(10) Scan parquet spark_catalog.default.store_sales +(8) Scan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] @@ -90,216 +84,232 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] Condition : (isnotnull(ss_customer_sk#6) AND isnotnull(ss_item_sk#5)) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -(13) BroadcastExchange -Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [2]: [ca_state#2, c_customer_sk#3] +Right output [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [c_customer_sk#3], [ss_customer_sk#6], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 7] -Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] +(12) CometProject Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7], [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] -(16) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#9] +(13) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_month_seq#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] +ReadSchema: struct -(17) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(14) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#10] +Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date_sk#9)) + +(15) CometProject +Input [2]: [d_date_sk#9, d_month_seq#10] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(16) CometBroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: [d_date_sk#9] + +(17) CometBroadcastHashJoin +Left output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] +Right output [1]: [d_date_sk#9] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight -(18) Project [codegen id : 7] -Output [2]: [ca_state#2, ss_item_sk#5] +(18) CometProject Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] +Arguments: [ca_state#2, ss_item_sk#5], [ca_state#2, ss_item_sk#5] + +(19) ColumnarToRow [codegen id : 4] +Input [2]: [ca_state#2, ss_item_sk#5] -(19) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#10, i_current_price#11, i_category#12] +(20) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#13, i_current_price#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] ReadSchema: struct -(20) CometFilter -Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] -Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10)) +(21) CometFilter +Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] +Condition : ((isnotnull(i_current_price#14) AND isnotnull(i_category#15)) AND isnotnull(i_item_sk#13)) -(21) ColumnarToRow [codegen id : 6] -Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] +(22) ColumnarToRow [codegen id : 3] +Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] -(22) Scan parquet spark_catalog.default.item -Output [2]: [i_current_price#13, i_category#14] +(23) Scan parquet spark_catalog.default.item +Output [2]: [i_current_price#16, i_category#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct -(23) CometFilter -Input [2]: [i_current_price#13, i_category#14] -Condition : isnotnull(i_category#14) +(24) CometFilter +Input [2]: [i_current_price#16, i_category#17] +Condition : isnotnull(i_category#17) -(24) CometHashAggregate -Input [2]: [i_current_price#13, i_category#14] -Keys [1]: [i_category#14] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] +(25) CometHashAggregate +Input [2]: [i_current_price#16, i_category#17] +Keys [1]: [i_category#17] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#16))] -(25) ColumnarToRow [codegen id : 4] -Input [3]: [i_category#14, sum#15, count#16] +(26) ColumnarToRow [codegen id : 1] +Input [3]: [i_category#17, sum#18, count#19] -(26) Exchange -Input [3]: [i_category#14, sum#15, count#16] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(27) Exchange +Input [3]: [i_category#17, sum#18, count#19] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(27) HashAggregate [codegen id : 5] -Input [3]: [i_category#14, sum#15, count#16] -Keys [1]: [i_category#14] -Functions [1]: [avg(UnscaledValue(i_current_price#13))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#17] -Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#17 / 100.0) as decimal(11,6)) AS avg(i_current_price)#18, i_category#14] +(28) HashAggregate [codegen id : 2] +Input [3]: [i_category#17, sum#18, count#19] +Keys [1]: [i_category#17] +Functions [1]: [avg(UnscaledValue(i_current_price#16))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#16))#20] +Results [2]: [cast((avg(UnscaledValue(i_current_price#16))#20 / 100.0) as decimal(11,6)) AS avg(i_current_price)#21, i_category#17] -(28) Filter [codegen id : 5] -Input [2]: [avg(i_current_price)#18, i_category#14] -Condition : isnotnull(avg(i_current_price)#18) +(29) Filter [codegen id : 2] +Input [2]: [avg(i_current_price)#21, i_category#17] +Condition : isnotnull(avg(i_current_price)#21) -(29) BroadcastExchange -Input [2]: [avg(i_current_price)#18, i_category#14] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] +(30) BroadcastExchange +Input [2]: [avg(i_current_price)#21, i_category#17] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=2] -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_category#12] -Right keys [1]: [i_category#14] +(31) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_category#15] +Right keys [1]: [i_category#17] Join type: Inner -Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#18)) +Join condition: (cast(i_current_price#14 as decimal(14,7)) > (1.2 * avg(i_current_price)#21)) -(31) Project [codegen id : 6] -Output [1]: [i_item_sk#10] -Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#18, i_category#14] +(32) Project [codegen id : 3] +Output [1]: [i_item_sk#13] +Input [5]: [i_item_sk#13, i_current_price#14, i_category#15, avg(i_current_price)#21, i_category#17] -(32) BroadcastExchange -Input [1]: [i_item_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(33) BroadcastExchange +Input [1]: [i_item_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 7] +(34) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_item_sk#5] -Right keys [1]: [i_item_sk#10] +Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(34) Project [codegen id : 7] +(35) Project [codegen id : 4] Output [1]: [ca_state#2] -Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#10] +Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#13] -(35) HashAggregate [codegen id : 7] +(36) HashAggregate [codegen id : 4] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#19] -Results [2]: [ca_state#2, count#20] +Aggregate Attributes [1]: [count#22] +Results [2]: [ca_state#2, count#23] -(36) Exchange -Input [2]: [ca_state#2, count#20] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(37) Exchange +Input [2]: [ca_state#2, count#23] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(37) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#20] +(38) HashAggregate [codegen id : 5] +Input [2]: [ca_state#2, count#23] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#21] -Results [3]: [ca_state#2 AS state#22, count(1)#21 AS cnt#23, ca_state#2] +Aggregate Attributes [1]: [count(1)#24] +Results [3]: [ca_state#2 AS state#25, count(1)#24 AS cnt#26, ca_state#2] -(38) Filter [codegen id : 8] -Input [3]: [state#22, cnt#23, ca_state#2] -Condition : (cnt#23 >= 10) +(39) Filter [codegen id : 5] +Input [3]: [state#25, cnt#26, ca_state#2] +Condition : (cnt#26 >= 10) -(39) TakeOrderedAndProject -Input [3]: [state#22, cnt#23, ca_state#2] -Arguments: 100, [cnt#23 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#22, cnt#23] +(40) TakeOrderedAndProject +Input [3]: [state#25, cnt#26, ca_state#2] +Arguments: 100, [cnt#26 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#25, cnt#26] ===== Subqueries ===== -Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (44) -+- * ColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan parquet spark_catalog.default.date_dim (40) +Subquery:1 Hosting operator id = 8 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) -(40) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#9, d_month_seq#24] +(41) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_month_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter -Input [2]: [d_date_sk#9, d_month_seq#24] -Condition : ((isnotnull(d_month_seq#24) AND (d_month_seq#24 = Subquery scalar-subquery#25, [id=#26])) AND isnotnull(d_date_sk#9)) +(42) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#10] +Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = Subquery scalar-subquery#11, [id=#12])) AND isnotnull(d_date_sk#9)) -(42) CometProject -Input [2]: [d_date_sk#9, d_month_seq#24] +(43) CometProject +Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(43) ColumnarToRow [codegen id : 1] +(44) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(44) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#25, [id=#26] -* HashAggregate (51) -+- Exchange (50) - +- * ColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan parquet spark_catalog.default.date_dim (45) +Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* HashAggregate (52) ++- Exchange (51) + +- * ColumnarToRow (50) + +- CometHashAggregate (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(45) Scan parquet spark_catalog.default.date_dim +(46) Scan parquet spark_catalog.default.date_dim Output [3]: [d_month_seq#27, d_year#28, d_moy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct -(46) CometFilter +(47) CometFilter Input [3]: [d_month_seq#27, d_year#28, d_moy#29] Condition : (((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2000)) AND (d_moy#29 = 1)) -(47) CometProject +(48) CometProject Input [3]: [d_month_seq#27, d_year#28, d_moy#29] Arguments: [d_month_seq#27], [d_month_seq#27] -(48) CometHashAggregate +(49) CometHashAggregate Input [1]: [d_month_seq#27] Keys [1]: [d_month_seq#27] Functions: [] -(49) ColumnarToRow [codegen id : 1] +(50) ColumnarToRow [codegen id : 1] Input [1]: [d_month_seq#27] -(50) Exchange +(51) Exchange Input [1]: [d_month_seq#27] -Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(d_month_seq#27, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(51) HashAggregate [codegen id : 2] +(52) HashAggregate [codegen id : 2] Input [1]: [d_month_seq#27] Keys [1]: [d_month_seq#27] Functions: [] Aggregate Attributes: [] Results [1]: [d_month_seq#27] +Subquery:3 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index 57ef6ebd5f..9607ab8874 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -1,62 +1,57 @@ TakeOrderedAndProject [cnt,ca_state,state] - WholeStageCodegen (8) + WholeStageCodegen (5) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] InputAdapter Exchange [ca_state] #1 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - Project [ca_state,c_customer_sk] - BroadcastHashJoin [ca_address_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ca_state,ss_item_sk] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ca_state,ss_item_sk,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometProject [ca_state,c_customer_sk] + CometBroadcastHashJoin [ca_address_sk,c_current_addr_sk] CometFilter [ca_address_sk] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [c_current_addr_sk,c_customer_sk] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk,ss_item_sk] - CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_month_seq,d_date_sk] - Subquery #2 - WholeStageCodegen (2) - HashAggregate [d_month_seq] - InputAdapter - Exchange [d_month_seq] #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #2 + CometFilter [c_current_addr_sk,c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange #3 + CometFilter [ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + Subquery #2 + WholeStageCodegen (2) + HashAggregate [d_month_seq] + InputAdapter + Exchange [d_month_seq] #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (3) Project [i_item_sk] BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] ColumnarToRow @@ -64,13 +59,13 @@ TakeOrderedAndProject [cnt,ca_state,state] CometFilter [i_current_price,i_category,i_item_sk] CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) + BroadcastExchange #8 + WholeStageCodegen (2) Filter [avg(i_current_price)] HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter - Exchange [i_category] #8 - WholeStageCodegen (4) + Exchange [i_category] #9 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometHashAggregate [i_category,i_current_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 626c5688cd..4088fb4098 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -216,7 +216,7 @@ Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_numbe (7) CometBroadcastHashJoin Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner +Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft (8) CometProject Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] @@ -725,7 +725,7 @@ Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_nu (117) CometBroadcastHashJoin Left output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] Right output [2]: [sr_item_sk#119, sr_ticket_number#120] -Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner +Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner, BuildLeft (118) CometProject Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index 2c32227f85..76545b83ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -1,71 +1,72 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * Filter (66) - +- Window (65) - +- * Sort (64) - +- Exchange (63) - +- Union (62) - :- * HashAggregate (21) - : +- Exchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * ColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (16) - : +- * ColumnarToRow (15) - : +- CometFilter (14) - : +- CometScan parquet spark_catalog.default.item (13) - :- * HashAggregate (26) - : +- Exchange (25) - : +- * HashAggregate (24) - : +- * HashAggregate (23) - : +- ReusedExchange (22) - :- * HashAggregate (31) - : +- Exchange (30) - : +- * HashAggregate (29) - : +- * HashAggregate (28) - : +- ReusedExchange (27) - :- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * HashAggregate (33) - : +- ReusedExchange (32) - :- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * HashAggregate (38) - : +- ReusedExchange (37) - :- * HashAggregate (46) - : +- Exchange (45) - : +- * HashAggregate (44) - : +- * HashAggregate (43) - : +- ReusedExchange (42) - :- * HashAggregate (51) - : +- Exchange (50) - : +- * HashAggregate (49) - : +- * HashAggregate (48) - : +- ReusedExchange (47) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * HashAggregate (53) - : +- ReusedExchange (52) - +- * HashAggregate (61) - +- Exchange (60) - +- * HashAggregate (59) - +- * HashAggregate (58) - +- ReusedExchange (57) +TakeOrderedAndProject (68) ++- * Filter (67) + +- Window (66) + +- * Sort (65) + +- Exchange (64) + +- Union (63) + :- * HashAggregate (22) + : +- Exchange (21) + : +- * ColumnarToRow (20) + : +- CometHashAggregate (19) + : +- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (11) + : : +- CometFilter (10) + : : +- CometScan parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.item (14) + :- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * HashAggregate (24) + : +- ReusedExchange (23) + :- * HashAggregate (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- * HashAggregate (29) + : +- ReusedExchange (28) + :- * HashAggregate (37) + : +- Exchange (36) + : +- * HashAggregate (35) + : +- * HashAggregate (34) + : +- ReusedExchange (33) + :- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * HashAggregate (39) + : +- ReusedExchange (38) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * HashAggregate (44) + : +- ReusedExchange (43) + :- * HashAggregate (52) + : +- Exchange (51) + : +- * HashAggregate (50) + : +- * HashAggregate (49) + : +- ReusedExchange (48) + :- * HashAggregate (57) + : +- Exchange (56) + : +- * HashAggregate (55) + : +- * HashAggregate (54) + : +- ReusedExchange (53) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * HashAggregate (59) + +- ReusedExchange (58) (1) Scan parquet spark_catalog.default.store_sales @@ -80,372 +81,377 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 72] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(4) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None +(5) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(6) CometBroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(7) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] +(7) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] +Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight + +(8) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] + +(9) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#12, s_store_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) - -(9) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#12] +(10) CometFilter +Input [2]: [s_store_sk#12, s_store_id#13] +Condition : isnotnull(s_store_sk#12) -(10) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [2]: [s_store_sk#12, s_store_id#13] +Arguments: [s_store_sk#12, s_store_id#13] -(11) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] +Right output [2]: [s_store_sk#12, s_store_id#13] +Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight -(12) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#12] +(13) CometProject +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#13] +Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13] -(13) Scan parquet spark_catalog.default.item -Output [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +(14) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(14) CometFilter -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Condition : isnotnull(i_item_sk#13) +(15) CometFilter +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Condition : isnotnull(i_item_sk#14) -(15) ColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +(16) CometBroadcastExchange +Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -(16) BroadcastExchange -Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(17) CometBroadcastHashJoin +Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13] +Right output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [ss_item_sk#1], [i_item_sk#14], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(18) CometProject +Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18] -(18) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] - -(19) HashAggregate [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_brand#14, i_class#15, i_category#16, i_product_name#17] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +(19) CometHashAggregate +Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] -(20) Exchange -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(20) ColumnarToRow [codegen id : 1] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#19, isEmpty#20] + +(21) Exchange +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#19, isEmpty#20] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(21) HashAggregate [codegen id : 5] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +(22) HashAggregate [codegen id : 2] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#19, isEmpty#20] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21] +Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21 as decimal(38,2)) AS sumsales#22] -(22) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25] +(23) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#23, isEmpty#24] -(23) HashAggregate [codegen id : 10] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +(24) HashAggregate [codegen id : 4] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#23, isEmpty#24] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] - -(24) HashAggregate [codegen id : 10] -Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sumsales#26] -Keys [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#27, isEmpty#28] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] - -(25) Exchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(26) HashAggregate [codegen id : 11] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum#29, isEmpty#30] -Keys [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#31] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, null AS s_store_id#32, sum(sumsales#26)#31 AS sumsales#33] - -(27) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35] - -(28) HashAggregate [codegen id : 16] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21] +Results [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21 AS sumsales#25] + +(25) HashAggregate [codegen id : 4] +Input [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, sumsales#25] +Keys [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10] +Functions [1]: [partial_sum(sumsales#25)] +Aggregate Attributes [2]: [sum#26, isEmpty#27] +Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, sum#28, isEmpty#29] + +(26) Exchange +Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, sum#28, isEmpty#29] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(27) HashAggregate [codegen id : 5] +Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, sum#28, isEmpty#29] +Keys [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10] +Functions [1]: [sum(sumsales#25)] +Aggregate Attributes [1]: [sum(sumsales#25)#30] +Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, null AS s_store_id#31, sum(sumsales#25)#30 AS sumsales#32] + +(28) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#33, isEmpty#34] + +(29) HashAggregate [codegen id : 7] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#33, isEmpty#34] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] - -(29) HashAggregate [codegen id : 16] -Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sumsales#26] -Keys [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] - -(30) Exchange -Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(31) HashAggregate [codegen id : 17] -Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum#38, isEmpty#39] -Keys [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#40] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, null AS d_moy#41, null AS s_store_id#42, sum(sumsales#26)#40 AS sumsales#43] - -(32) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45] - -(33) HashAggregate [codegen id : 22] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21] +Results [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21 AS sumsales#25] + +(30) HashAggregate [codegen id : 7] +Input [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, sumsales#25] +Keys [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11] +Functions [1]: [partial_sum(sumsales#25)] +Aggregate Attributes [2]: [sum#35, isEmpty#36] +Results [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, sum#37, isEmpty#38] + +(31) Exchange +Input [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, sum#37, isEmpty#38] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(32) HashAggregate [codegen id : 8] +Input [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, sum#37, isEmpty#38] +Keys [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11] +Functions [1]: [sum(sumsales#25)] +Aggregate Attributes [1]: [sum(sumsales#25)#39] +Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, null AS d_moy#40, null AS s_store_id#41, sum(sumsales#25)#39 AS sumsales#42] + +(33) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#43, isEmpty#44] + +(34) HashAggregate [codegen id : 10] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#43, isEmpty#44] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] - -(34) HashAggregate [codegen id : 22] -Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sumsales#26] -Keys [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] - -(35) Exchange -Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, 5), ENSURE_REQUIREMENTS, [plan_id=6] - -(36) HashAggregate [codegen id : 23] -Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum#48, isEmpty#49] -Keys [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#50] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, null AS d_qoy#51, null AS d_moy#52, null AS s_store_id#53, sum(sumsales#26)#50 AS sumsales#54] - -(37) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56] - -(38) HashAggregate [codegen id : 28] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21] +Results [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21 AS sumsales#25] + +(35) HashAggregate [codegen id : 10] +Input [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, sumsales#25] +Keys [5]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9] +Functions [1]: [partial_sum(sumsales#25)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, sum#47, isEmpty#48] + +(36) Exchange +Input [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(37) HashAggregate [codegen id : 11] +Input [7]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, sum#47, isEmpty#48] +Keys [5]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9] +Functions [1]: [sum(sumsales#25)] +Aggregate Attributes [1]: [sum(sumsales#25)#49] +Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, null AS d_qoy#50, null AS d_moy#51, null AS s_store_id#52, sum(sumsales#25)#49 AS sumsales#53] + +(38) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#54, isEmpty#55] + +(39) HashAggregate [codegen id : 13] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#54, isEmpty#55] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] - -(39) HashAggregate [codegen id : 28] -Input [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sumsales#26] -Keys [4]: [i_category#16, i_class#15, i_brand#14, i_product_name#17] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] - -(40) Exchange -Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(41) HashAggregate [codegen id : 29] -Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum#59, isEmpty#60] -Keys [4]: [i_category#16, i_class#15, i_brand#14, i_product_name#17] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#61] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, null AS d_year#62, null AS d_qoy#63, null AS d_moy#64, null AS s_store_id#65, sum(sumsales#26)#61 AS sumsales#66] - -(42) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68] - -(43) HashAggregate [codegen id : 34] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21] +Results [5]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21 AS sumsales#25] + +(40) HashAggregate [codegen id : 13] +Input [5]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sumsales#25] +Keys [4]: [i_category#17, i_class#16, i_brand#15, i_product_name#18] +Functions [1]: [partial_sum(sumsales#25)] +Aggregate Attributes [2]: [sum#56, isEmpty#57] +Results [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sum#58, isEmpty#59] + +(41) Exchange +Input [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sum#58, isEmpty#59] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, i_product_name#18, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(42) HashAggregate [codegen id : 14] +Input [6]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, sum#58, isEmpty#59] +Keys [4]: [i_category#17, i_class#16, i_brand#15, i_product_name#18] +Functions [1]: [sum(sumsales#25)] +Aggregate Attributes [1]: [sum(sumsales#25)#60] +Results [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, null AS d_year#61, null AS d_qoy#62, null AS d_moy#63, null AS s_store_id#64, sum(sumsales#25)#60 AS sumsales#65] + +(43) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#66, isEmpty#67] + +(44) HashAggregate [codegen id : 16] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#66, isEmpty#67] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [4]: [i_category#16, i_class#15, i_brand#14, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] - -(44) HashAggregate [codegen id : 34] -Input [4]: [i_category#16, i_class#15, i_brand#14, sumsales#26] -Keys [3]: [i_category#16, i_class#15, i_brand#14] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#69, isEmpty#70] -Results [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] - -(45) Exchange -Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] -Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, 5), ENSURE_REQUIREMENTS, [plan_id=8] - -(46) HashAggregate [codegen id : 35] -Input [5]: [i_category#16, i_class#15, i_brand#14, sum#71, isEmpty#72] -Keys [3]: [i_category#16, i_class#15, i_brand#14] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#73] -Results [9]: [i_category#16, i_class#15, i_brand#14, null AS i_product_name#74, null AS d_year#75, null AS d_qoy#76, null AS d_moy#77, null AS s_store_id#78, sum(sumsales#26)#73 AS sumsales#79] - -(47) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81] - -(48) HashAggregate [codegen id : 40] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21] +Results [4]: [i_category#17, i_class#16, i_brand#15, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21 AS sumsales#25] + +(45) HashAggregate [codegen id : 16] +Input [4]: [i_category#17, i_class#16, i_brand#15, sumsales#25] +Keys [3]: [i_category#17, i_class#16, i_brand#15] +Functions [1]: [partial_sum(sumsales#25)] +Aggregate Attributes [2]: [sum#68, isEmpty#69] +Results [5]: [i_category#17, i_class#16, i_brand#15, sum#70, isEmpty#71] + +(46) Exchange +Input [5]: [i_category#17, i_class#16, i_brand#15, sum#70, isEmpty#71] +Arguments: hashpartitioning(i_category#17, i_class#16, i_brand#15, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(47) HashAggregate [codegen id : 17] +Input [5]: [i_category#17, i_class#16, i_brand#15, sum#70, isEmpty#71] +Keys [3]: [i_category#17, i_class#16, i_brand#15] +Functions [1]: [sum(sumsales#25)] +Aggregate Attributes [1]: [sum(sumsales#25)#72] +Results [9]: [i_category#17, i_class#16, i_brand#15, null AS i_product_name#73, null AS d_year#74, null AS d_qoy#75, null AS d_moy#76, null AS s_store_id#77, sum(sumsales#25)#72 AS sumsales#78] + +(48) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#79, isEmpty#80] + +(49) HashAggregate [codegen id : 19] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#79, isEmpty#80] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [3]: [i_category#16, i_class#15, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] - -(49) HashAggregate [codegen id : 40] -Input [3]: [i_category#16, i_class#15, sumsales#26] -Keys [2]: [i_category#16, i_class#15] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#82, isEmpty#83] -Results [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] - -(50) Exchange -Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] -Arguments: hashpartitioning(i_category#16, i_class#15, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(51) HashAggregate [codegen id : 41] -Input [4]: [i_category#16, i_class#15, sum#84, isEmpty#85] -Keys [2]: [i_category#16, i_class#15] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#86] -Results [9]: [i_category#16, i_class#15, null AS i_brand#87, null AS i_product_name#88, null AS d_year#89, null AS d_qoy#90, null AS d_moy#91, null AS s_store_id#92, sum(sumsales#26)#86 AS sumsales#93] - -(52) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95] - -(53) HashAggregate [codegen id : 46] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21] +Results [3]: [i_category#17, i_class#16, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21 AS sumsales#25] + +(50) HashAggregate [codegen id : 19] +Input [3]: [i_category#17, i_class#16, sumsales#25] +Keys [2]: [i_category#17, i_class#16] +Functions [1]: [partial_sum(sumsales#25)] +Aggregate Attributes [2]: [sum#81, isEmpty#82] +Results [4]: [i_category#17, i_class#16, sum#83, isEmpty#84] + +(51) Exchange +Input [4]: [i_category#17, i_class#16, sum#83, isEmpty#84] +Arguments: hashpartitioning(i_category#17, i_class#16, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(52) HashAggregate [codegen id : 20] +Input [4]: [i_category#17, i_class#16, sum#83, isEmpty#84] +Keys [2]: [i_category#17, i_class#16] +Functions [1]: [sum(sumsales#25)] +Aggregate Attributes [1]: [sum(sumsales#25)#85] +Results [9]: [i_category#17, i_class#16, null AS i_brand#86, null AS i_product_name#87, null AS d_year#88, null AS d_qoy#89, null AS d_moy#90, null AS s_store_id#91, sum(sumsales#25)#85 AS sumsales#92] + +(53) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#93, isEmpty#94] + +(54) HashAggregate [codegen id : 22] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#93, isEmpty#94] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [2]: [i_category#16, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] - -(54) HashAggregate [codegen id : 46] -Input [2]: [i_category#16, sumsales#26] -Keys [1]: [i_category#16] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#96, isEmpty#97] -Results [3]: [i_category#16, sum#98, isEmpty#99] - -(55) Exchange -Input [3]: [i_category#16, sum#98, isEmpty#99] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(56) HashAggregate [codegen id : 47] -Input [3]: [i_category#16, sum#98, isEmpty#99] -Keys [1]: [i_category#16] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#100] -Results [9]: [i_category#16, null AS i_class#101, null AS i_brand#102, null AS i_product_name#103, null AS d_year#104, null AS d_qoy#105, null AS d_moy#106, null AS s_store_id#107, sum(sumsales#26)#100 AS sumsales#108] - -(57) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110] - -(58) HashAggregate [codegen id : 52] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110] -Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21] +Results [2]: [i_category#17, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21 AS sumsales#25] + +(55) HashAggregate [codegen id : 22] +Input [2]: [i_category#17, sumsales#25] +Keys [1]: [i_category#17] +Functions [1]: [partial_sum(sumsales#25)] +Aggregate Attributes [2]: [sum#95, isEmpty#96] +Results [3]: [i_category#17, sum#97, isEmpty#98] + +(56) Exchange +Input [3]: [i_category#17, sum#97, isEmpty#98] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(57) HashAggregate [codegen id : 23] +Input [3]: [i_category#17, sum#97, isEmpty#98] +Keys [1]: [i_category#17] +Functions [1]: [sum(sumsales#25)] +Aggregate Attributes [1]: [sum(sumsales#25)#99] +Results [9]: [i_category#17, null AS i_class#100, null AS i_brand#101, null AS i_product_name#102, null AS d_year#103, null AS d_qoy#104, null AS d_moy#105, null AS s_store_id#106, sum(sumsales#25)#99 AS sumsales#107] + +(58) ReusedExchange [Reuses operator id: 21] +Output [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#108, isEmpty#109] + +(59) HashAggregate [codegen id : 25] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sum#108, isEmpty#109] +Keys [8]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21] +Results [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#21 AS sumsales#25] -(59) HashAggregate [codegen id : 52] -Input [1]: [sumsales#26] +(60) HashAggregate [codegen id : 25] +Input [1]: [sumsales#25] Keys: [] -Functions [1]: [partial_sum(sumsales#26)] -Aggregate Attributes [2]: [sum#111, isEmpty#112] -Results [2]: [sum#113, isEmpty#114] +Functions [1]: [partial_sum(sumsales#25)] +Aggregate Attributes [2]: [sum#110, isEmpty#111] +Results [2]: [sum#112, isEmpty#113] -(60) Exchange -Input [2]: [sum#113, isEmpty#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] +(61) Exchange +Input [2]: [sum#112, isEmpty#113] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(61) HashAggregate [codegen id : 53] -Input [2]: [sum#113, isEmpty#114] +(62) HashAggregate [codegen id : 26] +Input [2]: [sum#112, isEmpty#113] Keys: [] -Functions [1]: [sum(sumsales#26)] -Aggregate Attributes [1]: [sum(sumsales#26)#115] -Results [9]: [null AS i_category#116, null AS i_class#117, null AS i_brand#118, null AS i_product_name#119, null AS d_year#120, null AS d_qoy#121, null AS d_moy#122, null AS s_store_id#123, sum(sumsales#26)#115 AS sumsales#124] +Functions [1]: [sum(sumsales#25)] +Aggregate Attributes [1]: [sum(sumsales#25)#114] +Results [9]: [null AS i_category#115, null AS i_class#116, null AS i_brand#117, null AS i_product_name#118, null AS d_year#119, null AS d_qoy#120, null AS d_moy#121, null AS s_store_id#122, sum(sumsales#25)#114 AS sumsales#123] -(62) Union +(63) Union -(63) Exchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=12] +(64) Exchange +Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#22] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(64) Sort [codegen id : 54] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +(65) Sort [codegen id : 27] +Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#22] +Arguments: [i_category#17 ASC NULLS FIRST, sumsales#22 DESC NULLS LAST], false, 0 -(65) Window -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#125], [i_category#16], [sumsales#23 DESC NULLS LAST] +(66) Window +Input [9]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#22] +Arguments: [rank(sumsales#22) windowspecdefinition(i_category#17, sumsales#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#124], [i_category#17], [sumsales#22 DESC NULLS LAST] -(66) Filter [codegen id : 55] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] -Condition : (rk#125 <= 100) +(67) Filter [codegen id : 28] +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#22, rk#124] +Condition : (rk#124 <= 100) -(67) TakeOrderedAndProject -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] -Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#125 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125] +(68) TakeOrderedAndProject +Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#22, rk#124] +Arguments: 100, [i_category#17 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_product_name#18 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#13 ASC NULLS FIRST, sumsales#22 ASC NULLS FIRST, rk#124 ASC NULLS FIRST], [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, sumsales#22, rk#124] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (72) -+- * ColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan parquet spark_catalog.default.date_dim (68) +BroadcastExchange (73) ++- * ColumnarToRow (72) + +- CometProject (71) + +- CometFilter (70) + +- CometScan parquet spark_catalog.default.date_dim (69) -(68) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10] +(69) Scan parquet spark_catalog.default.date_dim +Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(69) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#126) AND (d_month_seq#126 >= 1212)) AND (d_month_seq#126 <= 1223)) AND isnotnull(d_date_sk#7)) +(70) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) -(70) CometProject -Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(71) CometProject +Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] +Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(71) ColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +(72) ColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(72) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(73) BroadcastExchange +Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index cfac29f8a3..a3a99e312f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -1,27 +1,27 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (55) + WholeStageCodegen (28) Filter [rk] InputAdapter Window [sumsales,i_category] - WholeStageCodegen (54) + WholeStageCodegen (27) Sort [i_category,sumsales] InputAdapter Exchange [i_category] #1 Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] + CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk,ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -32,90 +32,84 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) + CometBroadcastExchange #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange #5 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + WholeStageCodegen (5) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - WholeStageCodegen (10) + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 + WholeStageCodegen (4) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - WholeStageCodegen (16) + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 + WholeStageCodegen (7) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) + WholeStageCodegen (11) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - WholeStageCodegen (22) + Exchange [i_category,i_class,i_brand,i_product_name,d_year] #9 + WholeStageCodegen (10) HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) + WholeStageCodegen (14) HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class,i_brand,i_product_name] #9 - WholeStageCodegen (28) + Exchange [i_category,i_class,i_brand,i_product_name] #10 + WholeStageCodegen (13) HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) + WholeStageCodegen (17) HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class,i_brand] #10 - WholeStageCodegen (34) + Exchange [i_category,i_class,i_brand] #11 + WholeStageCodegen (16) HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) + WholeStageCodegen (20) HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category,i_class] #11 - WholeStageCodegen (40) + Exchange [i_category,i_class] #12 + WholeStageCodegen (19) HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) + WholeStageCodegen (23) HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange [i_category] #12 - WholeStageCodegen (46) + Exchange [i_category] #13 + WholeStageCodegen (22) HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) + WholeStageCodegen (26) HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter - Exchange #13 - WholeStageCodegen (52) + Exchange #14 + WholeStageCodegen (25) HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 3a0ad3894e..9e148a8275 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -1,59 +1,61 @@ == Physical Plan == -TakeOrderedAndProject (55) -+- * Project (54) - +- Window (53) - +- * Sort (52) - +- Exchange (51) - +- * HashAggregate (50) - +- Exchange (49) - +- * HashAggregate (48) - +- Union (47) - :- * HashAggregate (36) - : +- Exchange (35) - : +- * HashAggregate (34) - : +- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (31) - : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : :- * ColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (29) - : +- * Project (28) - : +- * Filter (27) - : +- Window (26) - : +- * Sort (25) - : +- * HashAggregate (24) - : +- Exchange (23) - : +- * HashAggregate (22) - : +- * Project (21) - : +- * BroadcastHashJoin Inner BuildRight (20) - : :- * Project (18) - : : +- * BroadcastHashJoin Inner BuildRight (17) - : : :- * ColumnarToRow (12) - : : : +- CometFilter (11) - : : : +- CometScan parquet spark_catalog.default.store_sales (10) - : : +- BroadcastExchange (16) - : : +- * ColumnarToRow (15) - : : +- CometFilter (14) - : : +- CometScan parquet spark_catalog.default.store (13) - : +- ReusedExchange (19) - :- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * HashAggregate (38) - : +- ReusedExchange (37) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- ReusedExchange (42) +TakeOrderedAndProject (57) ++- * Project (56) + +- Window (55) + +- * Sort (54) + +- Exchange (53) + +- * HashAggregate (52) + +- Exchange (51) + +- * HashAggregate (50) + +- Union (49) + :- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * ColumnarToRow (9) + : : +- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- BroadcastExchange (33) + : +- * BroadcastHashJoin LeftSemi BuildRight (32) + : :- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store (10) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- Window (28) + : +- * Sort (27) + : +- * HashAggregate (26) + : +- Exchange (25) + : +- * ColumnarToRow (24) + : +- CometHashAggregate (23) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.store_sales (13) + : : +- CometBroadcastExchange (17) + : : +- CometFilter (16) + : : +- CometScan parquet spark_catalog.default.store (15) + : +- ReusedExchange (20) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * HashAggregate (40) + : +- ReusedExchange (39) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- ReusedExchange (44) (1) Scan parquet spark_catalog.default.store_sales @@ -68,296 +70,304 @@ ReadSchema: struct Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) -(3) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(7) Scan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] +(9) ColumnarToRow [codegen id : 5] +Input [2]: [ss_store_sk#1, ss_net_profit#2] + +(10) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) +(11) CometFilter +Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Condition : isnotnull(s_store_sk#7) -(9) ColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] +(12) ColumnarToRow [codegen id : 4] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(13) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) CometFilter -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(12) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +(14) CometFilter +Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_store_sk#10) -(13) Scan parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] +(15) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : isnotnull(s_store_sk#13) +(16) CometFilter +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) -(15) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#13, s_state#14] +(17) CometBroadcastExchange +Input [2]: [s_store_sk#14, s_state#15] +Arguments: [s_store_sk#14, s_state#15] -(16) BroadcastExchange -Input [2]: [s_store_sk#13, s_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(18) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] +Right output [2]: [s_store_sk#14, s_state#15] +Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None +(19) CometProject +Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#15] +Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15], [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] -(18) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#16] -(19) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#15] +(21) CometBroadcastHashJoin +Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15] +Right output [1]: [d_date_sk#16] +Arguments: [ss_sold_date_sk#12], [d_date_sk#16], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None +(22) CometProject +Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15, d_date_sk#16] +Arguments: [ss_net_profit#11, s_state#15], [ss_net_profit#11, s_state#15] + +(23) CometHashAggregate +Input [2]: [ss_net_profit#11, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] + +(24) ColumnarToRow [codegen id : 1] +Input [2]: [s_state#15, sum#17] -(21) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#14] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] - -(22) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#14] -Keys [1]: [s_state#14] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [s_state#14, sum#17] - -(23) Exchange -Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] - -(24) HashAggregate [codegen id : 5] -Input [2]: [s_state#14, sum#17] -Keys [1]: [s_state#14] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] -Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] - -(25) Sort [codegen id : 5] -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 - -(26) Window -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] - -(27) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] +(25) Exchange +Input [2]: [s_state#15, sum#17] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(26) HashAggregate [codegen id : 2] +Input [2]: [s_state#15, sum#17] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#18] +Results [3]: [s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#18,17,2) AS _w0#19, s_state#15] + +(27) Sort [codegen id : 2] +Input [3]: [s_state#15, _w0#19, s_state#15] +Arguments: [s_state#15 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 + +(28) Window +Input [3]: [s_state#15, _w0#19, s_state#15] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#15, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#15], [_w0#19 DESC NULLS LAST] + +(29) Filter [codegen id : 3] +Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] Condition : (ranking#20 <= 5) -(28) Project [codegen id : 6] -Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] +(30) Project [codegen id : 3] +Output [1]: [s_state#15] +Input [4]: [s_state#15, _w0#19, s_state#15, ranking#20] -(29) BroadcastExchange -Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] +(31) BroadcastExchange +Input [1]: [s_state#15] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(30) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [s_state#8] -Right keys [1]: [s_state#14] +(32) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [s_state#9] +Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(31) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +(33) BroadcastExchange +Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(32) BroadcastHashJoin [codegen id : 8] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] +Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(33) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_county#7, s_state#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] +(35) Project [codegen id : 5] +Output [3]: [ss_net_profit#2, s_county#8, s_state#9] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#9] -(34) HashAggregate [codegen id : 8] -Input [3]: [ss_net_profit#2, s_county#7, s_state#8] -Keys [2]: [s_state#8, s_county#7] +(36) HashAggregate [codegen id : 5] +Input [3]: [ss_net_profit#2, s_county#8, s_state#9] +Keys [2]: [s_state#9, s_county#8] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#21] -Results [3]: [s_state#8, s_county#7, sum#22] +Results [3]: [s_state#9, s_county#8, sum#22] -(35) Exchange -Input [3]: [s_state#8, s_county#7, sum#22] -Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(37) Exchange +Input [3]: [s_state#9, s_county#8, sum#22] +Arguments: hashpartitioning(s_state#9, s_county#8, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(36) HashAggregate [codegen id : 9] -Input [3]: [s_state#8, s_county#7, sum#22] -Keys [2]: [s_state#8, s_county#7] +(38) HashAggregate [codegen id : 6] +Input [3]: [s_state#9, s_county#8, sum#22] +Keys [2]: [s_state#9, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) as decimal(27,2)) AS total_sum#24, s_state#8, s_county#7, 0 AS g_state#25, 0 AS g_county#26, 0 AS lochierarchy#27] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) as decimal(27,2)) AS total_sum#24, s_state#9, s_county#8, 0 AS g_state#25, 0 AS g_county#26, 0 AS lochierarchy#27] -(37) ReusedExchange [Reuses operator id: 35] -Output [3]: [s_state#8, s_county#7, sum#28] +(39) ReusedExchange [Reuses operator id: 37] +Output [3]: [s_state#9, s_county#8, sum#28] -(38) HashAggregate [codegen id : 18] -Input [3]: [s_state#8, s_county#7, sum#28] -Keys [2]: [s_state#8, s_county#7] +(40) HashAggregate [codegen id : 12] +Input [3]: [s_state#9, s_county#8, sum#28] +Keys [2]: [s_state#9, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS total_sum#29, s_state#8] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS total_sum#29, s_state#9] -(39) HashAggregate [codegen id : 18] -Input [2]: [total_sum#29, s_state#8] -Keys [1]: [s_state#8] +(41) HashAggregate [codegen id : 12] +Input [2]: [total_sum#29, s_state#9] +Keys [1]: [s_state#9] Functions [1]: [partial_sum(total_sum#29)] Aggregate Attributes [2]: [sum#30, isEmpty#31] -Results [3]: [s_state#8, sum#32, isEmpty#33] +Results [3]: [s_state#9, sum#32, isEmpty#33] -(40) Exchange -Input [3]: [s_state#8, sum#32, isEmpty#33] -Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(42) Exchange +Input [3]: [s_state#9, sum#32, isEmpty#33] +Arguments: hashpartitioning(s_state#9, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(41) HashAggregate [codegen id : 19] -Input [3]: [s_state#8, sum#32, isEmpty#33] -Keys [1]: [s_state#8] +(43) HashAggregate [codegen id : 13] +Input [3]: [s_state#9, sum#32, isEmpty#33] +Keys [1]: [s_state#9] Functions [1]: [sum(total_sum#29)] Aggregate Attributes [1]: [sum(total_sum#29)#34] -Results [6]: [sum(total_sum#29)#34 AS total_sum#35, s_state#8, null AS s_county#36, 0 AS g_state#37, 1 AS g_county#38, 1 AS lochierarchy#39] +Results [6]: [sum(total_sum#29)#34 AS total_sum#35, s_state#9, null AS s_county#36, 0 AS g_state#37, 1 AS g_county#38, 1 AS lochierarchy#39] -(42) ReusedExchange [Reuses operator id: 35] -Output [3]: [s_state#8, s_county#7, sum#40] +(44) ReusedExchange [Reuses operator id: 37] +Output [3]: [s_state#9, s_county#8, sum#40] -(43) HashAggregate [codegen id : 28] -Input [3]: [s_state#8, s_county#7, sum#40] -Keys [2]: [s_state#8, s_county#7] +(45) HashAggregate [codegen id : 19] +Input [3]: [s_state#9, s_county#8, sum#40] +Keys [2]: [s_state#9, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS total_sum#29] -(44) HashAggregate [codegen id : 28] +(46) HashAggregate [codegen id : 19] Input [1]: [total_sum#29] Keys: [] Functions [1]: [partial_sum(total_sum#29)] Aggregate Attributes [2]: [sum#41, isEmpty#42] Results [2]: [sum#43, isEmpty#44] -(45) Exchange +(47) Exchange Input [2]: [sum#43, isEmpty#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(46) HashAggregate [codegen id : 29] +(48) HashAggregate [codegen id : 20] Input [2]: [sum#43, isEmpty#44] Keys: [] Functions [1]: [sum(total_sum#29)] Aggregate Attributes [1]: [sum(total_sum#29)#45] Results [6]: [sum(total_sum#29)#45 AS total_sum#46, null AS s_state#47, null AS s_county#48, 1 AS g_state#49, 1 AS g_county#50, 2 AS lochierarchy#51] -(47) Union +(49) Union -(48) HashAggregate [codegen id : 30] -Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +(50) HashAggregate [codegen id : 21] +Input [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] +Keys [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +Results [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] -(49) Exchange -Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -Arguments: hashpartitioning(total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [plan_id=8] +(51) Exchange +Input [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] +Arguments: hashpartitioning(total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(50) HashAggregate [codegen id : 31] -Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] -Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +(52) HashAggregate [codegen id : 22] +Input [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] +Keys [6]: [total_sum#24, s_state#9, s_county#8, g_state#25, g_county#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, CASE WHEN (g_county#26 = 0) THEN s_state#8 END AS _w0#52] +Results [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, CASE WHEN (g_county#26 = 0) THEN s_state#9 END AS _w0#52] -(51) Exchange -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] -Arguments: hashpartitioning(lochierarchy#27, _w0#52, 5), ENSURE_REQUIREMENTS, [plan_id=9] +(53) Exchange +Input [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, _w0#52] +Arguments: hashpartitioning(lochierarchy#27, _w0#52, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(52) Sort [codegen id : 32] -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] +(54) Sort [codegen id : 23] +Input [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, _w0#52] Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#52 ASC NULLS FIRST, total_sum#24 DESC NULLS LAST], false, 0 -(53) Window -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52] +(55) Window +Input [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, _w0#52] Arguments: [rank(total_sum#24) windowspecdefinition(lochierarchy#27, _w0#52, total_sum#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#53], [lochierarchy#27, _w0#52], [total_sum#24 DESC NULLS LAST] -(54) Project [codegen id : 33] -Output [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] -Input [6]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#52, rank_within_parent#53] +(56) Project [codegen id : 24] +Output [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, rank_within_parent#53] +Input [6]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, _w0#52, rank_within_parent#53] -(55) TakeOrderedAndProject -Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#53 ASC NULLS FIRST], [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#53] +(57) TakeOrderedAndProject +Input [5]: [total_sum#24, s_state#9, s_county#8, lochierarchy#27, rank_within_parent#53] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#9 END ASC NULLS FIRST, rank_within_parent#53 ASC NULLS FIRST], [total_sum#24, s_state#9, s_county#8, lochierarchy#27, rank_within_parent#53] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (60) -+- * ColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan parquet spark_catalog.default.date_dim (56) +BroadcastExchange (62) ++- * ColumnarToRow (61) + +- CometProject (60) + +- CometFilter (59) + +- CometScan parquet spark_catalog.default.date_dim (58) -(56) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#54] +(58) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(57) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#54] -Condition : (((isnotnull(d_month_seq#54) AND (d_month_seq#54 >= 1212)) AND (d_month_seq#54 <= 1223)) AND isnotnull(d_date_sk#5)) +(59) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(58) CometProject -Input [2]: [d_date_sk#5, d_month_seq#54] +(60) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(59) ColumnarToRow [codegen id : 1] +(61) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(60) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 663f828f46..9e72edbb4f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -1,32 +1,32 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (33) + WholeStageCodegen (24) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (32) + WholeStageCodegen (23) Sort [lochierarchy,_w0,total_sum] InputAdapter Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (31) + WholeStageCodegen (22) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] [_w0] InputAdapter Exchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (30) + WholeStageCodegen (21) HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] InputAdapter Union - WholeStageCodegen (9) + WholeStageCodegen (6) HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] InputAdapter Exchange [s_state,s_county] #3 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] Project [ss_net_profit,s_county,s_state] BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -37,62 +37,59 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) + BroadcastExchange #6 + WholeStageCodegen (4) BroadcastHashJoin [s_state,s_state] ColumnarToRow InputAdapter CometFilter [s_store_sk] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) + BroadcastExchange #7 + WholeStageCodegen (3) Project [s_state] Filter [ranking] InputAdapter Window [_w0,s_state] - WholeStageCodegen (5) + WholeStageCodegen (2) Sort [s_state,_w0] HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] InputAdapter - Exchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - ColumnarToRow - InputAdapter + Exchange [s_state] #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [s_state,ss_net_profit] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) + CometBroadcastExchange #9 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (13) HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter - Exchange [s_state] #9 - WholeStageCodegen (18) + Exchange [s_state] #10 + WholeStageCodegen (12) HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) + WholeStageCodegen (20) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] InputAdapter - Exchange #10 - WholeStageCodegen (28) + Exchange #11 + WholeStageCodegen (19) HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index c885738383..8bff19a727 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -1,74 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * SortMergeJoin LeftOuter (65) - :- * Sort (58) - : +- Exchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * ColumnarToRow (3) - : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * ColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * ColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * ColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * ColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * ColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * ColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * ColumnarToRow (53) - : +- CometFilter (52) - : +- CometScan parquet spark_catalog.default.promotion (51) - +- * Sort (64) - +- Exchange (63) - +- * ColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.catalog_returns (59) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- * Project (63) + +- * SortMergeJoin LeftOuter (62) + :- * Sort (55) + : +- Exchange (54) + : +- * Project (53) + : +- * BroadcastHashJoin LeftOuter BuildRight (52) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) + : : :- * ColumnarToRow (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (35) + : : : : +- CometBroadcastHashJoin (34) + : : : : :- CometProject (29) + : : : : : +- CometBroadcastHashJoin (28) + : : : : : :- CometProject (23) + : : : : : : +- CometBroadcastHashJoin (22) + : : : : : : :- CometProject (17) + : : : : : : : +- CometBroadcastHashJoin (16) + : : : : : : : :- CometProject (12) + : : : : : : : : +- CometBroadcastHashJoin (11) + : : : : : : : : :- CometProject (7) + : : : : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : : : : :- CometFilter (2) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- CometBroadcastExchange (5) + : : : : : : : : : +- CometFilter (4) + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (3) + : : : : : : : : +- CometBroadcastExchange (10) + : : : : : : : : +- CometFilter (9) + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (8) + : : : : : : : +- CometBroadcastExchange (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.item (13) + : : : : : : +- CometBroadcastExchange (21) + : : : : : : +- CometProject (20) + : : : : : : +- CometFilter (19) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) + : : : : : +- CometBroadcastExchange (27) + : : : : : +- CometProject (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan parquet spark_catalog.default.household_demographics (24) + : : : : +- CometBroadcastExchange (33) + : : : : +- CometProject (32) + : : : : +- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.date_dim (30) + : : : +- CometBroadcastExchange (38) + : : : +- CometFilter (37) + : : : +- CometScan parquet spark_catalog.default.date_dim (36) + : : +- BroadcastExchange (45) + : : +- * ColumnarToRow (44) + : : +- CometFilter (43) + : : +- CometScan parquet spark_catalog.default.date_dim (42) + : +- BroadcastExchange (51) + : +- * ColumnarToRow (50) + : +- CometFilter (49) + : +- CometScan parquet spark_catalog.default.promotion (48) + +- * Sort (61) + +- Exchange (60) + +- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.catalog_returns (56) (1) Scan parquet spark_catalog.default.catalog_sales @@ -83,10 +80,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -(7) BroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) +(6) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] +Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +(7) CometProject Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -(10) Scan parquet spark_catalog.default.warehouse +(8) Scan parquet spark_catalog.default.warehouse Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] PushedFilters: [IsNotNull(w_warehouse_sk)] ReadSchema: struct -(11) CometFilter +(9) CometFilter Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) ColumnarToRow [codegen id : 2] +(10) CometBroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [w_warehouse_sk#14, w_warehouse_name#15] -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] +(11) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +(12) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -(16) Scan parquet spark_catalog.default.item +(13) Scan parquet spark_catalog.default.item Output [2]: [i_item_sk#16, i_item_desc#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(17) CometFilter +(14) CometFilter Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#16, i_item_desc#17] - -(19) BroadcastExchange +(15) CometBroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#16, i_item_desc#17] -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +Right output [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(17) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] +Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(22) Scan parquet spark_catalog.default.customer_demographics +(18) Scan parquet spark_catalog.default.customer_demographics Output [2]: [cd_demo_sk#18, cd_marital_status#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_marital_status), EqualTo(cd_marital_status,M), IsNotNull(cd_demo_sk)] ReadSchema: struct -(23) CometFilter +(19) CometFilter Input [2]: [cd_demo_sk#18, cd_marital_status#19] Condition : ((isnotnull(cd_marital_status#19) AND (cd_marital_status#19 = M)) AND isnotnull(cd_demo_sk#18)) -(24) CometProject +(20) CometProject Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) ColumnarToRow [codegen id : 4] +(21) CometBroadcastExchange Input [1]: [cd_demo_sk#18] +Arguments: [cd_demo_sk#18] -(26) BroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None +(22) CometBroadcastHashJoin +Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [cd_demo_sk#18] +Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(23) CometProject Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] +Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(29) Scan parquet spark_catalog.default.household_demographics +(24) Scan parquet spark_catalog.default.household_demographics Output [2]: [hd_demo_sk#20, hd_buy_potential#21] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct -(30) CometFilter +(25) CometFilter Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Condition : ((isnotnull(hd_buy_potential#21) AND (hd_buy_potential#21 = 1001-5000 )) AND isnotnull(hd_demo_sk#20)) -(31) CometProject +(26) CometProject Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) ColumnarToRow [codegen id : 5] +(27) CometBroadcastExchange Input [1]: [hd_demo_sk#20] +Arguments: [hd_demo_sk#20] -(33) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(28) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [1]: [hd_demo_sk#20] +Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +(29) CometProject Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +(30) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +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 -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None +(31) CometFilter +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)) -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +(32) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] + +(33) CometBroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] + +(34) CometBroadcastHashJoin +Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight + +(35) CometProject Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(39) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] +(36) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#26, d_week_seq#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) +(37) CometFilter +Input [2]: [d_date_sk#26, d_week_seq#27] +Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) -(41) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] +(38) CometBroadcastExchange +Input [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_date_sk#26, d_week_seq#27] -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] +(39) CometBroadcastHashJoin +Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] +Right output [2]: [d_date_sk#26, d_week_seq#27] +Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None +(40) CometProject +Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] +Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] +(41) ColumnarToRow [codegen id : 3] +Input [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -(45) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] +(42) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_date#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) +(43) CometFilter +Input [2]: [d_date_sk#28, d_date#29] +Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) -(47) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] +(44) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_date#29] -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(45) BroadcastExchange +Input [2]: [d_date_sk#28, d_date#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(49) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#28] Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) +Join condition: (d_date#29 > date_add(d_date#23, 5)) -(50) Project [codegen id : 10] +(47) Project [codegen id : 3] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] +Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -(51) Scan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] +(48) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/promotion] PushedFilters: [IsNotNull(p_promo_sk)] ReadSchema: struct -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) +(49) CometFilter +Input [1]: [p_promo_sk#30] +Condition : isnotnull(p_promo_sk#30) -(53) ColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] +(50) ColumnarToRow [codegen id : 2] +Input [1]: [p_promo_sk#30] -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(51) BroadcastExchange +Input [1]: [p_promo_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(55) BroadcastHashJoin [codegen id : 10] +(52) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] +Right keys [1]: [p_promo_sk#30] Join type: LeftOuter Join condition: None -(56) Project [codegen id : 10] +(53) Project [codegen id : 3] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] +Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -(57) Exchange +(54) Exchange Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(58) Sort [codegen id : 11] +(55) Sort [codegen id : 4] Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 -(59) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +(56) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) +(57) CometFilter +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] +(58) CometProject +Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] +Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] -(62) ColumnarToRow [codegen id : 12] -Input [2]: [cr_item_sk#30, cr_order_number#31] +(59) ColumnarToRow [codegen id : 5] +Input [2]: [cr_item_sk#31, cr_order_number#32] -(63) Exchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, [plan_id=10] +(60) Exchange +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(64) Sort [codegen id : 13] -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST], false, 0 +(61) Sort [codegen id : 6] +Input [2]: [cr_item_sk#31, cr_order_number#32] +Arguments: [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST], false, 0 -(65) SortMergeJoin [codegen id : 14] +(62) SortMergeJoin [codegen id : 7] Left keys [2]: [cs_item_sk#4, cs_order_number#6] -Right keys [2]: [cr_item_sk#30, cr_order_number#31] +Right keys [2]: [cr_item_sk#31, cr_order_number#32] Join type: LeftOuter Join condition: None -(66) Project [codegen id : 14] +(63) Project [codegen id : 7] Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -(67) HashAggregate [codegen id : 14] +(64) HashAggregate [codegen id : 7] Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Aggregate Attributes [1]: [count#34] +Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] -(68) Exchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] +(65) Exchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(69) HashAggregate [codegen id : 15] -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +(66) HashAggregate [codegen id : 8] +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#35] Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#35 AS no_promo#36, count(1)#35 AS promo#37, count(1)#35 AS total_cnt#38] +Aggregate Attributes [1]: [count(1)#36] +Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#36 AS no_promo#37, count(1)#36 AS promo#38, count(1)#36 AS total_cnt#39] -(70) TakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] -Arguments: 100, [total_cnt#38 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] +(67) TakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#37, promo#38, total_cnt#39] +Arguments: 100, [total_cnt#39 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#37, promo#38, total_cnt#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * ColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan parquet spark_catalog.default.date_dim (71) +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(71) Scan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +(68) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] 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 -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] -Condition : ((((isnotnull(d_year#39) AND (d_year#39 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(69) CometFilter +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)) -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +(70) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) ColumnarToRow [codegen id : 1] +(71) ColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(75) BroadcastExchange +(72) BroadcastExchange Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt index 17fc9dee7e..bea1fd4a16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt @@ -1,38 +1,38 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] - WholeStageCodegen (15) + WholeStageCodegen (8) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] InputAdapter Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - WholeStageCodegen (14) + WholeStageCodegen (7) HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] Project [w_warehouse_name,i_item_desc,d_week_seq] SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (4) Sort [cs_item_sk,cs_order_number] InputAdapter Exchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) + WholeStageCodegen (3) 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] BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - ColumnarToRow - InputAdapter + ColumnarToRow + InputAdapter + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + CometBroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + CometProject [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + CometBroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -43,72 +43,50 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_year,d_date_sk,d_week_seq,d_date] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] - CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_marital_status,cd_demo_sk] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - ColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_buy_potential,hd_demo_sk] - CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - ColumnarToRow - InputAdapter - CometFilter [d_week_seq,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometBroadcastExchange #4 + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometBroadcastExchange #5 + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometBroadcastExchange #7 + CometProject [cd_demo_sk] + CometFilter [cd_marital_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometBroadcastExchange #8 + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + CometBroadcastExchange #9 + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometBroadcastExchange #10 + CometFilter [d_week_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) + BroadcastExchange #11 + WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) + BroadcastExchange #12 + WholeStageCodegen (2) ColumnarToRow InputAdapter CometFilter [p_promo_sk] CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (6) Sort [cr_item_sk,cr_order_number] InputAdapter - Exchange [cr_item_sk,cr_order_number] #12 - WholeStageCodegen (12) + Exchange [cr_item_sk,cr_order_number] #13 + WholeStageCodegen (5) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index ad8c33ca1d..661c552f21 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -2,74 +2,74 @@ TakeOrderedAndProject (71) +- * Project (70) +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * BroadcastHashJoin Inner BuildRight (33) - : : :- * Filter (16) - : : : +- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * ColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (32) - : : +- * HashAggregate (31) - : : +- Exchange (30) - : : +- * HashAggregate (29) - : : +- * Project (28) - : : +- * BroadcastHashJoin Inner BuildRight (27) - : : :- * Project (25) - : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : :- * ColumnarToRow (19) - : : : : +- CometFilter (18) - : : : : +- CometScan parquet spark_catalog.default.customer (17) - : : : +- BroadcastExchange (23) - : : : +- * ColumnarToRow (22) - : : : +- CometFilter (21) - : : : +- CometScan parquet spark_catalog.default.store_sales (20) - : : +- ReusedExchange (26) - : +- BroadcastExchange (50) - : +- * Filter (49) - : +- * HashAggregate (48) - : +- Exchange (47) - : +- * HashAggregate (46) - : +- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * ColumnarToRow (36) - : : : +- CometFilter (35) - : : : +- CometScan parquet spark_catalog.default.customer (34) - : : +- BroadcastExchange (40) - : : +- * ColumnarToRow (39) - : : +- CometFilter (38) - : : +- CometScan parquet spark_catalog.default.web_sales (37) - : +- ReusedExchange (43) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Filter (17) + : : : +- * HashAggregate (16) + : : : +- Exchange (15) + : : : +- * ColumnarToRow (14) + : : : +- CometHashAggregate (13) + : : : +- CometProject (12) + : : : +- CometBroadcastHashJoin (11) + : : : :- CometProject (7) + : : : : +- CometBroadcastHashJoin (6) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- CometBroadcastExchange (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : +- CometBroadcastExchange (10) + : : : +- CometFilter (9) + : : : +- CometScan parquet spark_catalog.default.date_dim (8) + : : +- BroadcastExchange (34) + : : +- * HashAggregate (33) + : : +- Exchange (32) + : : +- * ColumnarToRow (31) + : : +- CometHashAggregate (30) + : : +- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer (18) + : : : +- CometBroadcastExchange (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- CometBroadcastExchange (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.date_dim (25) + : +- BroadcastExchange (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * ColumnarToRow (47) + : +- CometHashAggregate (46) + : +- CometProject (45) + : +- CometBroadcastHashJoin (44) + : :- CometProject (42) + : : +- CometBroadcastHashJoin (41) + : : :- CometFilter (37) + : : : +- CometScan parquet spark_catalog.default.customer (36) + : : +- CometBroadcastExchange (40) + : : +- CometFilter (39) + : : +- CometScan parquet spark_catalog.default.web_sales (38) + : +- ReusedExchange (43) +- BroadcastExchange (68) +- * HashAggregate (67) +- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * ColumnarToRow (55) - : : +- CometFilter (54) - : : +- CometScan parquet spark_catalog.default.customer (53) - : +- BroadcastExchange (59) - : +- * ColumnarToRow (58) - : +- CometFilter (57) - : +- CometScan parquet spark_catalog.default.web_sales (56) - +- ReusedExchange (62) + +- * ColumnarToRow (65) + +- CometHashAggregate (64) + +- CometProject (63) + +- CometBroadcastHashJoin (62) + :- CometProject (60) + : +- CometBroadcastHashJoin (59) + : :- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.customer (54) + : +- CometBroadcastExchange (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (61) (1) Scan parquet spark_catalog.default.customer @@ -83,10 +83,7 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] Condition : isnotnull(ss_customer_sk#5) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -(7) BroadcastExchange -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Right output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: [c_customer_sk#1], [ss_customer_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +(7) CometProject Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] -(10) ReusedExchange [Reuses operator id: 75] +(8) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#9, d_year#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#9, d_year#10] +Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) + +(10) CometBroadcastExchange +Input [2]: [d_date_sk#9, d_year#10] +Arguments: [d_date_sk#9, d_year#10] -(12) Project [codegen id : 3] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] +(11) CometBroadcastHashJoin +Left output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +Right output [2]: [d_date_sk#9, d_year#10] +Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight + +(12) CometProject Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10] +Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] -(13) HashAggregate [codegen id : 3] +(13) CometHashAggregate Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum#11] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -(14) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(14) ColumnarToRow [codegen id : 1] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#11] + +(15) Exchange +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#11] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 16] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] +(16) HashAggregate [codegen id : 8] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#11] Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] -Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#12] +Results [2]: [c_customer_id#2 AS customer_id#13, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#12,17,2) AS year_total#14] -(16) Filter [codegen id : 16] -Input [2]: [customer_id#14, year_total#15] -Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) +(17) Filter [codegen id : 8] +Input [2]: [customer_id#13, year_total#14] +Condition : (isnotnull(year_total#14) AND (year_total#14 > 0.00)) -(17) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +(18) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(18) CometFilter -Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] -Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) - -(19) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +(19) CometFilter +Input [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_customer_id#16)) (20) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (21) CometFilter -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#20) +Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Condition : isnotnull(ss_customer_sk#19) -(22) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +(22) CometBroadcastExchange +Input [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Arguments: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] -(23) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(23) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18] +Right output [3]: [ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Arguments: [c_customer_sk#15], [ss_customer_sk#19], Inner, BuildRight -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#16] -Right keys [1]: [ss_customer_sk#20] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] -Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] - -(26) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#24, d_year#25] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None +(24) CometProject +Input [7]: [c_customer_sk#15, c_customer_id#16, c_first_name#17, c_last_name#18, ss_customer_sk#19, ss_net_paid#20, ss_sold_date_sk#21] +Arguments: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21], [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21] -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] - -(29) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] -Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] -Aggregate Attributes [1]: [sum#26] -Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] - -(30) Exchange -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] - -(31) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] -Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] -Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] -Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] - -(32) BroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#23, d_year#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct -(33) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#28] +(26) CometFilter +Input [2]: [d_date_sk#23, d_year#24] +Condition : (((isnotnull(d_year#24) AND (d_year#24 = 2002)) AND d_year#24 IN (2001,2002)) AND isnotnull(d_date_sk#23)) + +(27) CometBroadcastExchange +Input [2]: [d_date_sk#23, d_year#24] +Arguments: [d_date_sk#23, d_year#24] + +(28) CometBroadcastHashJoin +Left output [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21] +Right output [2]: [d_date_sk#23, d_year#24] +Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight + +(29) CometProject +Input [7]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, ss_sold_date_sk#21, d_date_sk#23, d_year#24] +Arguments: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24], [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] + +(30) CometHashAggregate +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, ss_net_paid#20, d_year#24] +Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#20))] + +(31) ColumnarToRow [codegen id : 2] +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#25] + +(32) Exchange +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#25] +Arguments: hashpartitioning(c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(33) HashAggregate [codegen id : 3] +Input [5]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24, sum#25] +Keys [4]: [c_customer_id#16, c_first_name#17, c_last_name#18, d_year#24] +Functions [1]: [sum(UnscaledValue(ss_net_paid#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#20))#12] +Results [4]: [c_customer_id#16 AS customer_id#26, c_first_name#17 AS customer_first_name#27, c_last_name#18 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#20))#12,17,2) AS year_total#29] + +(34) BroadcastExchange +Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(35) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#26] Join type: Inner Join condition: None -(34) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +(36) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#30, c_customer_id#31, c_first_name#32, c_last_name#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(35) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) - -(36) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +(37) CometFilter +Input [4]: [c_customer_sk#30, c_customer_id#31, c_first_name#32, c_last_name#33] +Condition : (isnotnull(c_customer_sk#30) AND isnotnull(c_customer_id#31)) -(37) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(38) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(38) CometFilter -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_bill_customer_sk#36) +(39) CometFilter +Input [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_bill_customer_sk#34) -(39) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(40) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +Arguments: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] -(40) BroadcastExchange -Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(41) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#30, c_customer_id#31, c_first_name#32, c_last_name#33] +Right output [3]: [ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +Arguments: [c_customer_sk#30], [ws_bill_customer_sk#34], Inner, BuildRight -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#32] -Right keys [1]: [ws_bill_customer_sk#36] -Join type: Inner -Join condition: None +(42) CometProject +Input [7]: [c_customer_sk#30, c_customer_id#31, c_first_name#32, c_last_name#33, ws_bill_customer_sk#34, ws_net_paid#35, ws_sold_date_sk#36] +Arguments: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36], [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36] -(42) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] -Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +(43) ReusedExchange [Reuses operator id: 10] +Output [2]: [d_date_sk#38, d_year#39] -(43) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#40, d_year#41] +(44) CometBroadcastHashJoin +Left output [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36] +Right output [2]: [d_date_sk#38, d_year#39] +Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None +(45) CometProject +Input [7]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39] +Arguments: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39], [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39] + +(46) CometHashAggregate +Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, ws_net_paid#35, d_year#39] +Keys [4]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#35))] + +(47) ColumnarToRow [codegen id : 4] +Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] -(45) Project [codegen id : 10] -Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] - -(46) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] -Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] -Aggregate Attributes [1]: [sum#42] -Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] - -(47) Exchange -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=7] - -(48) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] -Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] -Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] -Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] - -(49) Filter [codegen id : 11] -Input [2]: [customer_id#45, year_total#46] -Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) - -(50) BroadcastExchange -Input [2]: [customer_id#45, year_total#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#45] +(48) Exchange +Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] +Arguments: hashpartitioning(c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(49) HashAggregate [codegen id : 5] +Input [5]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39, sum#40] +Keys [4]: [c_customer_id#31, c_first_name#32, c_last_name#33, d_year#39] +Functions [1]: [sum(UnscaledValue(ws_net_paid#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#35))#41] +Results [2]: [c_customer_id#31 AS customer_id#42, MakeDecimal(sum(UnscaledValue(ws_net_paid#35))#41,17,2) AS year_total#43] + +(50) Filter [codegen id : 5] +Input [2]: [customer_id#42, year_total#43] +Condition : (isnotnull(year_total#43) AND (year_total#43 > 0.00)) + +(51) BroadcastExchange +Input [2]: [customer_id#42, year_total#43] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(52) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#42] Join type: Inner Join condition: None -(52) Project [codegen id : 16] -Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] -Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] +(53) Project [codegen id : 8] +Output [7]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#43] +Input [8]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#42, year_total#43] -(53) Scan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +(54) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct -(54) CometFilter -Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] -Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) - -(55) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +(55) CometFilter +Input [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] +Condition : (isnotnull(c_customer_sk#44) AND isnotnull(c_customer_id#45)) (56) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ws_sold_date_sk#50), dynamicpruningexpression(ws_sold_date_sk#50 IN dynamicpruning#51)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (57) CometFilter -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_bill_customer_sk#51) +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +Condition : isnotnull(ws_bill_customer_sk#48) -(58) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +(58) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +Arguments: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] -(59) BroadcastExchange -Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(59) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47] +Right output [3]: [ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +Arguments: [c_customer_sk#44], [ws_bill_customer_sk#48], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#47] -Right keys [1]: [ws_bill_customer_sk#51] -Join type: Inner -Join condition: None +(60) CometProject +Input [7]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, ws_bill_customer_sk#48, ws_net_paid#49, ws_sold_date_sk#50] +Arguments: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50], [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50] -(61) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] -Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +(61) ReusedExchange [Reuses operator id: 27] +Output [2]: [d_date_sk#52, d_year#53] -(62) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#55, d_year#56] +(62) CometBroadcastHashJoin +Left output [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50] +Right output [2]: [d_date_sk#52, d_year#53] +Arguments: [ws_sold_date_sk#50], [d_date_sk#52], Inner, BuildRight -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#53] -Right keys [1]: [d_date_sk#55] -Join type: Inner -Join condition: None +(63) CometProject +Input [7]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, ws_sold_date_sk#50, d_date_sk#52, d_year#53] +Arguments: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53], [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53] -(64) Project [codegen id : 14] -Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] +(64) CometHashAggregate +Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, ws_net_paid#49, d_year#53] +Keys [4]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#49))] -(65) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] -Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] -Aggregate Attributes [1]: [sum#57] -Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +(65) ColumnarToRow [codegen id : 6] +Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] (66) Exchange -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] +Arguments: hashpartitioning(c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(67) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] -Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] -Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#52))#44] -Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#52))#44,17,2) AS year_total#60] +(67) HashAggregate [codegen id : 7] +Input [5]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53, sum#54] +Keys [4]: [c_customer_id#45, c_first_name#46, c_last_name#47, d_year#53] +Functions [1]: [sum(UnscaledValue(ws_net_paid#49))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#49))#41] +Results [2]: [c_customer_id#45 AS customer_id#55, MakeDecimal(sum(UnscaledValue(ws_net_paid#49))#41,17,2) AS year_total#56] (68) BroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +Input [2]: [customer_id#55, year_total#56] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] -(69) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#14] -Right keys [1]: [customer_id#59] +(69) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [customer_id#13] +Right keys [1]: [customer_id#55] Join type: Inner -Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) +Join condition: (CASE WHEN (year_total#43 > 0.00) THEN (year_total#56 / year_total#43) END > CASE WHEN (year_total#14 > 0.00) THEN (year_total#29 / year_total#14) END) -(70) Project [codegen id : 16] -Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] +(70) Project [codegen id : 8] +Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] +Input [9]: [customer_id#13, year_total#14, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#43, customer_id#55, year_total#56] (71) TakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] +Arguments: 100, [customer_first_name#27 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_last_name#28 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (75) +- * ColumnarToRow (74) +- CometFilter (73) @@ -443,9 +439,9 @@ Input [2]: [d_date_sk#9, d_year#10] (75) BroadcastExchange Input [2]: [d_date_sk#9, d_year#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#22 BroadcastExchange (79) +- * ColumnarToRow (78) +- CometFilter (77) @@ -453,25 +449,25 @@ BroadcastExchange (79) (76) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +Output [2]: [d_date_sk#23, d_year#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (77) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) +Input [2]: [d_date_sk#23, d_year#24] +Condition : (((isnotnull(d_year#24) AND (d_year#24 = 2002)) AND d_year#24 IN (2001,2002)) AND isnotnull(d_date_sk#23)) (78) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#24, d_year#25] +Input [2]: [d_date_sk#23, d_year#24] (79) BroadcastExchange -Input [2]: [d_date_sk#24, d_year#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Input [2]: [d_date_sk#23, d_year#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#50 IN dynamicpruning#22 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt index 26989b0c00..283fe5bde8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - WholeStageCodegen (16) + WholeStageCodegen (8) Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] @@ -9,114 +9,98 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] InputAdapter Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter + CometBroadcastExchange #2 + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange #4 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_customer_sk] + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + CometBroadcastExchange #7 CometFilter [ss_customer_sk] CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_customer_id] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometFilter [ss_customer_sk] - CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [d_year,d_date_sk] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #9 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #10 + WholeStageCodegen (5) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #11 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 + CometBroadcastExchange #12 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year] #4 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) + BroadcastExchange #13 + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] InputAdapter - Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #14 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] CometFilter [c_customer_sk,c_customer_id] CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - ColumnarToRow - InputAdapter - CometFilter [ws_bill_customer_sk] - CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 + CometBroadcastExchange #15 + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 14a9459aa9..564e409285 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -1,133 +1,136 @@ == Physical Plan == -TakeOrderedAndProject (129) -+- * Project (128) - +- * SortMergeJoin Inner (127) - :- * Sort (71) - : +- Exchange (70) - : +- * Filter (69) - : +- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- * HashAggregate (65) - : +- Exchange (64) - : +- * HashAggregate (63) - : +- Union (62) - : :- * Project (23) - : : +- * SortMergeJoin LeftOuter (22) - : : :- * Sort (15) - : : : +- Exchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * ColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- * Sort (21) - : : +- Exchange (20) - : : +- * ColumnarToRow (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : :- * Project (42) - : : +- * SortMergeJoin LeftOuter (41) - : : :- * Sort (34) - : : : +- Exchange (33) - : : : +- * Project (32) - : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : :- * Project (29) - : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : :- * ColumnarToRow (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) - : : : : +- ReusedExchange (27) - : : : +- ReusedExchange (30) - : : +- * Sort (40) - : : +- Exchange (39) - : : +- * ColumnarToRow (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometScan parquet spark_catalog.default.store_returns (35) - : +- * Project (61) - : +- * SortMergeJoin LeftOuter (60) - : :- * Sort (53) - : : +- Exchange (52) - : : +- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Project (48) - : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : :- * ColumnarToRow (45) - : : : : +- CometFilter (44) - : : : : +- CometScan parquet spark_catalog.default.web_sales (43) - : : : +- ReusedExchange (46) - : : +- ReusedExchange (49) - : +- * Sort (59) - : +- Exchange (58) - : +- * ColumnarToRow (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan parquet spark_catalog.default.web_returns (54) - +- * Sort (126) - +- Exchange (125) - +- * Filter (124) - +- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- * HashAggregate (120) - +- Exchange (119) - +- * HashAggregate (118) - +- Union (117) - :- * Project (86) - : +- * SortMergeJoin LeftOuter (85) - : :- * Sort (82) - : : +- Exchange (81) - : : +- * Project (80) - : : +- * BroadcastHashJoin Inner BuildRight (79) - : : :- * Project (77) - : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : :- * ColumnarToRow (74) - : : : : +- CometFilter (73) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) - : : : +- ReusedExchange (75) - : : +- ReusedExchange (78) - : +- * Sort (84) - : +- ReusedExchange (83) - :- * Project (101) - : +- * SortMergeJoin LeftOuter (100) - : :- * Sort (97) - : : +- Exchange (96) - : : +- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * ColumnarToRow (89) - : : : : +- CometFilter (88) - : : : : +- CometScan parquet spark_catalog.default.store_sales (87) - : : : +- ReusedExchange (90) - : : +- ReusedExchange (93) - : +- * Sort (99) - : +- ReusedExchange (98) - +- * Project (116) - +- * SortMergeJoin LeftOuter (115) - :- * Sort (112) - : +- Exchange (111) - : +- * Project (110) - : +- * BroadcastHashJoin Inner BuildRight (109) - : :- * Project (107) - : : +- * BroadcastHashJoin Inner BuildRight (106) - : : :- * ColumnarToRow (104) - : : : +- CometFilter (103) - : : : +- CometScan parquet spark_catalog.default.web_sales (102) - : : +- ReusedExchange (105) - : +- ReusedExchange (108) - +- * Sort (114) - +- ReusedExchange (113) +TakeOrderedAndProject (132) ++- * Project (131) + +- * SortMergeJoin Inner (130) + :- * Sort (72) + : +- Exchange (71) + : +- * Filter (70) + : +- * HashAggregate (69) + : +- Exchange (68) + : +- * HashAggregate (67) + : +- * HashAggregate (66) + : +- Exchange (65) + : +- * HashAggregate (64) + : +- Union (63) + : :- * Project (24) + : : +- * SortMergeJoin LeftOuter (23) + : : :- * Sort (16) + : : : +- Exchange (15) + : : : +- * ColumnarToRow (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.item (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.date_dim (9) + : : +- * Sort (22) + : : +- Exchange (21) + : : +- * ColumnarToRow (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : :- * Project (43) + : : +- * SortMergeJoin LeftOuter (42) + : : :- * Sort (35) + : : : +- Exchange (34) + : : : +- * ColumnarToRow (33) + : : : +- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometProject (29) + : : : : +- CometBroadcastHashJoin (28) + : : : : :- CometFilter (26) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (25) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- * Sort (41) + : : +- Exchange (40) + : : +- * ColumnarToRow (39) + : : +- CometProject (38) + : : +- CometFilter (37) + : : +- CometScan parquet spark_catalog.default.store_returns (36) + : +- * Project (62) + : +- * SortMergeJoin LeftOuter (61) + : :- * Sort (54) + : : +- Exchange (53) + : : +- * ColumnarToRow (52) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometProject (48) + : : : +- CometBroadcastHashJoin (47) + : : : :- CometFilter (45) + : : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- * Sort (60) + : +- Exchange (59) + : +- * ColumnarToRow (58) + : +- CometProject (57) + : +- CometFilter (56) + : +- CometScan parquet spark_catalog.default.web_returns (55) + +- * Sort (129) + +- Exchange (128) + +- * Filter (127) + +- * HashAggregate (126) + +- Exchange (125) + +- * HashAggregate (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- Union (120) + :- * Project (89) + : +- * SortMergeJoin LeftOuter (88) + : :- * Sort (85) + : : +- Exchange (84) + : : +- * ColumnarToRow (83) + : : +- CometProject (82) + : : +- CometBroadcastHashJoin (81) + : : :- CometProject (77) + : : : +- CometBroadcastHashJoin (76) + : : : :- CometFilter (74) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (73) + : : : +- ReusedExchange (75) + : : +- CometBroadcastExchange (80) + : : +- CometFilter (79) + : : +- CometScan parquet spark_catalog.default.date_dim (78) + : +- * Sort (87) + : +- ReusedExchange (86) + :- * Project (104) + : +- * SortMergeJoin LeftOuter (103) + : :- * Sort (100) + : : +- Exchange (99) + : : +- * ColumnarToRow (98) + : : +- CometProject (97) + : : +- CometBroadcastHashJoin (96) + : : :- CometProject (94) + : : : +- CometBroadcastHashJoin (93) + : : : :- CometFilter (91) + : : : : +- CometScan parquet spark_catalog.default.store_sales (90) + : : : +- ReusedExchange (92) + : : +- ReusedExchange (95) + : +- * Sort (102) + : +- ReusedExchange (101) + +- * Project (119) + +- * SortMergeJoin LeftOuter (118) + :- * Sort (115) + : +- Exchange (114) + : +- * ColumnarToRow (113) + : +- CometProject (112) + : +- CometBroadcastHashJoin (111) + : :- CometProject (109) + : : +- CometBroadcastHashJoin (108) + : : :- CometFilter (106) + : : : +- CometScan parquet spark_catalog.default.web_sales (105) + : : +- ReusedExchange (107) + : +- ReusedExchange (110) + +- * Sort (117) + +- ReusedExchange (116) (1) Scan parquet spark_catalog.default.catalog_sales @@ -142,99 +145,106 @@ ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) -(6) CometProject +(5) CometProject Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) ColumnarToRow [codegen id : 1] +(6) CometBroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(8) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None +(7) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +(8) CometProject Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(11) ReusedExchange [Reuses operator id: 133] +(9) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None +(10) CometFilter +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +(11) CometBroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: [d_date_sk#13, d_year#14] + +(12) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Right output [2]: [d_date_sk#13, d_year#14] +Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight + +(13) CometProject Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] +Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] + +(14) ColumnarToRow [codegen id : 1] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -(14) Exchange +(15) Exchange Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) Sort [codegen id : 4] +(16) Sort [codegen id : 2] Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 -(16) Scan parquet spark_catalog.default.catalog_returns +(17) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(17) CometFilter +(18) CometFilter Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) -(18) CometProject +(19) CometProject Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(19) ColumnarToRow [codegen id : 5] +(20) ColumnarToRow [codegen id : 3] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(20) Exchange +(21) Exchange Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(21) Sort [codegen id : 6] +(22) Sort [codegen id : 4] Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] Arguments: [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 7] +(23) SortMergeJoin [codegen id : 5] Left keys [2]: [cs_order_number#2, cs_item_sk#1] Right keys [2]: [cr_order_number#16, cr_item_sk#15] Join type: LeftOuter Join condition: None -(23) Project [codegen id : 7] +(24) Project [codegen id : 5] Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -(24) Scan parquet spark_catalog.default.store_sales +(25) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] @@ -242,84 +252,82 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(25) CometFilter +(26) CometFilter Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#22) -(26) ColumnarToRow [codegen id : 10] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(27) ReusedExchange [Reuses operator id: 8] +(27) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(28) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#22] -Right keys [1]: [i_item_sk#28] -Join type: Inner -Join condition: None +(28) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight -(29) Project [codegen id : 10] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +(29) CometProject Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -(30) ReusedExchange [Reuses operator id: 133] +(30) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#33, d_year#34] -(31) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#33] -Join type: Inner -Join condition: None +(31) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Right output [2]: [d_date_sk#33, d_year#34] +Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight -(32) Project [codegen id : 10] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +(32) CometProject Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] +Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] + +(33) ColumnarToRow [codegen id : 6] +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -(33) Exchange +(34) Exchange Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(34) Sort [codegen id : 11] +(35) Sort [codegen id : 7] Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] Arguments: [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST], false, 0 -(35) Scan parquet spark_catalog.default.store_returns +(36) Scan parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(36) CometFilter +(37) CometFilter Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) -(37) CometProject +(38) CometProject Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(38) ColumnarToRow [codegen id : 12] +(39) ColumnarToRow [codegen id : 8] Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(39) Exchange +(40) Exchange Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(40) Sort [codegen id : 13] +(41) Sort [codegen id : 9] Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] Arguments: [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST], false, 0 -(41) SortMergeJoin [codegen id : 14] +(42) SortMergeJoin [codegen id : 10] Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] Join type: LeftOuter Join condition: None -(42) Project [codegen id : 14] +(43) Project [codegen id : 10] Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -(43) Scan parquet spark_catalog.default.web_sales +(44) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] @@ -327,134 +335,132 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(44) CometFilter +(45) CometFilter Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] Condition : isnotnull(ws_item_sk#42) -(45) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] - -(46) ReusedExchange [Reuses operator id: 8] +(46) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(47) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#42] -Right keys [1]: [i_item_sk#48] -Join type: Inner -Join condition: None +(47) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight -(48) Project [codegen id : 17] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +(48) CometProject Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -(49) ReusedExchange [Reuses operator id: 133] +(49) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#53, d_year#54] -(50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#46] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(50) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Right output [2]: [d_date_sk#53, d_year#54] +Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight -(51) Project [codegen id : 17] -Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +(51) CometProject Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] +Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -(52) Exchange +(52) ColumnarToRow [codegen id : 11] Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(53) Sort [codegen id : 18] +(53) Exchange +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(54) Sort [codegen id : 12] Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] Arguments: [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST], false, 0 -(54) Scan parquet spark_catalog.default.web_returns +(55) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(55) CometFilter +(56) CometFilter Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) -(56) CometProject +(57) CometProject Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(57) ColumnarToRow [codegen id : 19] +(58) ColumnarToRow [codegen id : 13] Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(58) Exchange +(59) Exchange Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(59) Sort [codegen id : 20] +(60) Sort [codegen id : 14] Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] Arguments: [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST], false, 0 -(60) SortMergeJoin [codegen id : 21] +(61) SortMergeJoin [codegen id : 15] Left keys [2]: [ws_order_number#43, ws_item_sk#42] Right keys [2]: [wr_order_number#56, wr_item_sk#55] Join type: LeftOuter Join condition: None -(61) Project [codegen id : 21] +(62) Project [codegen id : 15] Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -(62) Union +(63) Union -(63) HashAggregate [codegen id : 22] +(64) HashAggregate [codegen id : 16] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(64) Exchange +(65) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(65) HashAggregate [codegen id : 23] +(66) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -(66) HashAggregate [codegen id : 23] +(67) HashAggregate [codegen id : 17] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum#62, sum#63] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -(67) Exchange +(68) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(68) HashAggregate [codegen id : 24] +(69) HashAggregate [codegen id : 18] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] -(69) Filter [codegen id : 24] +(70) Filter [codegen id : 18] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] Condition : isnotnull(sales_cnt#68) -(70) Exchange +(71) Exchange Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] -(71) Sort [codegen id : 25] +(72) Sort [codegen id : 19] Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 -(72) Scan parquet spark_catalog.default.catalog_sales +(73) Scan parquet spark_catalog.default.catalog_sales Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Batched: true Location: InMemoryFileIndex [] @@ -462,65 +468,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(73) CometFilter +(74) CometFilter Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] Condition : isnotnull(cs_item_sk#70) -(74) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] - -(75) ReusedExchange [Reuses operator id: 8] +(75) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(76) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#70] -Right keys [1]: [i_item_sk#76] -Join type: Inner -Join condition: None +(76) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Right output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70], [i_item_sk#76], Inner, BuildRight -(77) Project [codegen id : 28] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +(77) CometProject Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] -(78) ReusedExchange [Reuses operator id: 137] +(78) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#81, d_year#82] +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 -(79) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#74] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None +(79) CometFilter +Input [2]: [d_date_sk#81, d_year#82] +Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(80) Project [codegen id : 28] -Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +(80) CometBroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] +Arguments: [d_date_sk#81, d_year#82] + +(81) CometBroadcastHashJoin +Left output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Right output [2]: [d_date_sk#81, d_year#82] +Arguments: [cs_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight + +(82) CometProject Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] +Arguments: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82], [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] + +(83) ColumnarToRow [codegen id : 20] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -(81) Exchange +(84) Exchange Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] -Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(82) Sort [codegen id : 29] +(85) Sort [codegen id : 21] Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 -(83) ReusedExchange [Reuses operator id: 20] +(86) ReusedExchange [Reuses operator id: 21] Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(84) Sort [codegen id : 31] +(87) Sort [codegen id : 23] Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] Arguments: [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST], false, 0 -(85) SortMergeJoin [codegen id : 32] +(88) SortMergeJoin [codegen id : 24] Left keys [2]: [cs_order_number#71, cs_item_sk#70] Right keys [2]: [cr_order_number#84, cr_item_sk#83] Join type: LeftOuter Join condition: None -(86) Project [codegen id : 32] +(89) Project [codegen id : 24] Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#20, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#21] Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(87) Scan parquet spark_catalog.default.store_sales +(90) Scan parquet spark_catalog.default.store_sales Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] @@ -528,65 +544,63 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(88) CometFilter +(91) CometFilter Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] Condition : isnotnull(ss_item_sk#87) -(89) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] - -(90) ReusedExchange [Reuses operator id: 8] +(92) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(91) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#87] -Right keys [1]: [i_item_sk#93] -Join type: Inner -Join condition: None +(93) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +Right output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +Arguments: [ss_item_sk#87], [i_item_sk#93], Inner, BuildRight -(92) Project [codegen id : 35] -Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(94) CometProject Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -(93) ReusedExchange [Reuses operator id: 137] +(95) ReusedExchange [Reuses operator id: 80] Output [2]: [d_date_sk#98, d_year#99] -(94) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#91] -Right keys [1]: [d_date_sk#98] -Join type: Inner -Join condition: None +(96) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +Right output [2]: [d_date_sk#98, d_year#99] +Arguments: [ss_sold_date_sk#91], [d_date_sk#98], Inner, BuildRight -(95) Project [codegen id : 35] -Output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +(97) CometProject Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] +Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -(96) Exchange +(98) ColumnarToRow [codegen id : 25] Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(97) Sort [codegen id : 36] +(99) Exchange +Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(100) Sort [codegen id : 26] Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] Arguments: [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST], false, 0 -(98) ReusedExchange [Reuses operator id: 39] +(101) ReusedExchange [Reuses operator id: 40] Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -(99) Sort [codegen id : 38] +(102) Sort [codegen id : 28] Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] Arguments: [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST], false, 0 -(100) SortMergeJoin [codegen id : 39] +(103) SortMergeJoin [codegen id : 29] Left keys [2]: [ss_ticket_number#88, ss_item_sk#87] Right keys [2]: [sr_ticket_number#101, sr_item_sk#100] Join type: LeftOuter Join condition: None -(101) Project [codegen id : 39] +(104) Project [codegen id : 29] Output [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#40, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#41] Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -(102) Scan parquet spark_catalog.default.web_sales +(105) Scan parquet spark_catalog.default.web_sales Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] @@ -594,186 +608,184 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(103) CometFilter +(106) CometFilter Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Condition : isnotnull(ws_item_sk#104) -(104) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] - -(105) ReusedExchange [Reuses operator id: 8] +(107) ReusedExchange [Reuses operator id: 6] Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(106) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#104] -Right keys [1]: [i_item_sk#110] -Join type: Inner -Join condition: None +(108) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104], [i_item_sk#110], Inner, BuildRight -(107) Project [codegen id : 42] -Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +(109) CometProject Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(108) ReusedExchange [Reuses operator id: 137] +(110) ReusedExchange [Reuses operator id: 80] Output [2]: [d_date_sk#115, d_year#116] -(109) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#108] -Right keys [1]: [d_date_sk#115] -Join type: Inner -Join condition: None +(111) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Right output [2]: [d_date_sk#115, d_year#116] +Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner, BuildRight -(110) Project [codegen id : 42] -Output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +(112) CometProject Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] + +(113) ColumnarToRow [codegen id : 30] +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -(111) Exchange +(114) Exchange Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(112) Sort [codegen id : 43] +(115) Sort [codegen id : 31] Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] Arguments: [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST], false, 0 -(113) ReusedExchange [Reuses operator id: 58] +(116) ReusedExchange [Reuses operator id: 59] Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(114) Sort [codegen id : 45] +(117) Sort [codegen id : 33] Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] Arguments: [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST], false, 0 -(115) SortMergeJoin [codegen id : 46] +(118) SortMergeJoin [codegen id : 34] Left keys [2]: [ws_order_number#105, ws_item_sk#104] Right keys [2]: [wr_order_number#118, wr_item_sk#117] Join type: LeftOuter Join condition: None -(116) Project [codegen id : 46] +(119) Project [codegen id : 34] Output [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#60, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#61] Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(117) Union +(120) Union -(118) HashAggregate [codegen id : 47] +(121) HashAggregate [codegen id : 35] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(119) Exchange +(122) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(120) HashAggregate [codegen id : 48] +(123) HashAggregate [codegen id : 36] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Functions: [] Aggregate Attributes: [] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] -(121) HashAggregate [codegen id : 48] +(124) HashAggregate [codegen id : 36] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#20, sales_amt#21] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum#62, sum#121] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -(122) Exchange +(125) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] -Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=14] -(123) HashAggregate [codegen id : 49] +(126) HashAggregate [codegen id : 37] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#64, sum#122] Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#20)#66 AS sales_cnt#123, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#124] -(124) Filter [codegen id : 49] +(127) Filter [codegen id : 37] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] Condition : isnotnull(sales_cnt#123) -(125) Exchange +(128) Exchange Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] +Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] -(126) Sort [codegen id : 50] +(129) Sort [codegen id : 38] Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 -(127) SortMergeJoin [codegen id : 51] +(130) SortMergeJoin [codegen id : 39] Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] Join type: Inner Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#123 as decimal(17,2))) < 0.90000000000000000000) -(128) Project [codegen id : 51] +(131) Project [codegen id : 39] Output [10]: [d_year#82 AS prev_year#125, d_year#14 AS year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#123 AS prev_yr_cnt#127, sales_cnt#68 AS curr_yr_cnt#128, (sales_cnt#68 - sales_cnt#123) AS sales_cnt_diff#129, (sales_amt#69 - sales_amt#124) AS sales_amt_diff#130] Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#123, sales_amt#124] -(129) TakeOrderedAndProject +(132) TakeOrderedAndProject Input [10]: [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] Arguments: 100, [sales_cnt_diff#129 ASC NULLS FIRST, sales_amt_diff#130 ASC NULLS FIRST], [prev_year#125, year#126, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#127, curr_yr_cnt#128, sales_cnt_diff#129, sales_amt_diff#130] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (133) -+- * ColumnarToRow (132) - +- CometFilter (131) - +- CometScan parquet spark_catalog.default.date_dim (130) +BroadcastExchange (136) ++- * ColumnarToRow (135) + +- CometFilter (134) + +- CometScan parquet spark_catalog.default.date_dim (133) -(130) Scan parquet spark_catalog.default.date_dim +(133) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#13, d_year#14] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(131) CometFilter +(134) CometFilter Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(132) ColumnarToRow [codegen id : 1] +(135) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(133) BroadcastExchange +(136) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] -Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 -BroadcastExchange (137) -+- * ColumnarToRow (136) - +- CometFilter (135) - +- CometScan parquet spark_catalog.default.date_dim (134) +Subquery:4 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (140) ++- * ColumnarToRow (139) + +- CometFilter (138) + +- CometScan parquet spark_catalog.default.date_dim (137) -(134) Scan parquet spark_catalog.default.date_dim +(137) Scan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#81, d_year#82] 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 -(135) CometFilter +(138) CometFilter Input [2]: [d_date_sk#81, d_year#82] Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) -(136) ColumnarToRow [codegen id : 1] +(139) ColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] -(137) BroadcastExchange +(140) BroadcastExchange Input [2]: [d_date_sk#81, d_year#82] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] -Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 90 Hosting Expression = ss_sold_date_sk#91 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 105 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#75 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt index 9939b2fe21..fb78d64b13 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt @@ -1,41 +1,41 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt] - WholeStageCodegen (51) + WholeStageCodegen (39) Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] InputAdapter - WholeStageCodegen (25) + WholeStageCodegen (19) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (24) + WholeStageCodegen (18) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - WholeStageCodegen (23) + WholeStageCodegen (17) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - WholeStageCodegen (22) + WholeStageCodegen (16) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (7) + WholeStageCodegen (5) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (2) Sort [cs_order_number,cs_item_sk] InputAdapter Exchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -45,196 +45,185 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + CometBroadcastExchange #6 + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometBroadcastExchange #7 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (4) Sort [cr_order_number,cr_item_sk] InputAdapter - Exchange [cr_order_number,cr_item_sk] #7 - WholeStageCodegen (5) + Exchange [cr_order_number,cr_item_sk] #8 + WholeStageCodegen (3) ColumnarToRow InputAdapter CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] CometFilter [cr_order_number,cr_item_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - WholeStageCodegen (14) + WholeStageCodegen (10) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (11) + WholeStageCodegen (7) Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #9 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter - WholeStageCodegen (13) + WholeStageCodegen (9) Sort [sr_ticket_number,sr_item_sk] InputAdapter - Exchange [sr_ticket_number,sr_item_sk] #9 - WholeStageCodegen (12) + Exchange [sr_ticket_number,sr_item_sk] #10 + WholeStageCodegen (8) ColumnarToRow InputAdapter CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] CometFilter [sr_ticket_number,sr_item_sk] CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - WholeStageCodegen (21) + WholeStageCodegen (15) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (18) + WholeStageCodegen (12) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (17) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ws_order_number,ws_item_sk] #11 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #7 InputAdapter - WholeStageCodegen (20) + WholeStageCodegen (14) Sort [wr_order_number,wr_item_sk] InputAdapter - Exchange [wr_order_number,wr_item_sk] #11 - WholeStageCodegen (19) + Exchange [wr_order_number,wr_item_sk] #12 + WholeStageCodegen (13) ColumnarToRow InputAdapter CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_order_number,wr_item_sk] CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] InputAdapter - WholeStageCodegen (50) + WholeStageCodegen (38) Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - WholeStageCodegen (49) + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + WholeStageCodegen (37) Filter [sales_cnt] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (48) + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + WholeStageCodegen (36) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - WholeStageCodegen (47) + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 + WholeStageCodegen (35) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] InputAdapter Union - WholeStageCodegen (32) + WholeStageCodegen (24) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] InputAdapter - WholeStageCodegen (29) + WholeStageCodegen (21) Sort [cs_order_number,cs_item_sk] InputAdapter - Exchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (28) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [cs_order_number,cs_item_sk] #16 + WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,i_item_sk] CometFilter [cs_item_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 + BroadcastExchange #17 WholeStageCodegen (1) ColumnarToRow InputAdapter CometFilter [d_year,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + CometBroadcastExchange #18 + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (31) + WholeStageCodegen (23) Sort [cr_order_number,cr_item_sk] InputAdapter - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - WholeStageCodegen (39) + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 + WholeStageCodegen (29) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] InputAdapter - WholeStageCodegen (36) + WholeStageCodegen (26) Sort [ss_ticket_number,ss_item_sk] InputAdapter - Exchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (35) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #19 + WholeStageCodegen (25) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter - WholeStageCodegen (38) + WholeStageCodegen (28) Sort [sr_ticket_number,sr_item_sk] InputAdapter - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - WholeStageCodegen (46) + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 + WholeStageCodegen (34) Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] InputAdapter - WholeStageCodegen (43) + WholeStageCodegen (31) Sort [ws_order_number,ws_item_sk] InputAdapter - Exchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (42) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + Exchange [ws_order_number,ws_item_sk] #20 + WholeStageCodegen (30) + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + ReusedExchange [d_date_sk,d_year] #18 InputAdapter - WholeStageCodegen (45) + WholeStageCodegen (33) Sort [wr_order_number,wr_item_sk] InputAdapter - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt index dd92e2a1cc..168e394dd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt @@ -1,102 +1,103 @@ == Physical Plan == -TakeOrderedAndProject (98) -+- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- Union (94) - :- * HashAggregate (83) - : +- Exchange (82) - : +- * HashAggregate (81) - : +- Union (80) - : :- * Project (30) - : : +- * BroadcastHashJoin LeftOuter BuildRight (29) - : : :- * HashAggregate (15) - : : : +- Exchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * ColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * ColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometScan parquet spark_catalog.default.store (7) - : : +- BroadcastExchange (28) - : : +- * HashAggregate (27) - : : +- Exchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * ColumnarToRow (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.store_returns (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (49) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (48) - : : :- BroadcastExchange (39) - : : : +- * HashAggregate (38) - : : : +- Exchange (37) - : : : +- * HashAggregate (36) - : : : +- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * ColumnarToRow (32) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) - : : : +- ReusedExchange (33) - : : +- * HashAggregate (47) - : : +- Exchange (46) - : : +- * HashAggregate (45) - : : +- * Project (44) - : : +- * BroadcastHashJoin Inner BuildRight (43) - : : :- * ColumnarToRow (41) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (40) - : : +- ReusedExchange (42) - : +- * Project (79) - : +- * BroadcastHashJoin LeftOuter BuildRight (78) - : :- * HashAggregate (64) - : : +- Exchange (63) - : : +- * HashAggregate (62) - : : +- * Project (61) - : : +- * BroadcastHashJoin Inner BuildRight (60) - : : :- * Project (55) - : : : +- * BroadcastHashJoin Inner BuildRight (54) - : : : :- * ColumnarToRow (52) - : : : : +- CometFilter (51) - : : : : +- CometScan parquet spark_catalog.default.web_sales (50) - : : : +- ReusedExchange (53) - : : +- BroadcastExchange (59) - : : +- * ColumnarToRow (58) - : : +- CometFilter (57) - : : +- CometScan parquet spark_catalog.default.web_page (56) - : +- BroadcastExchange (77) - : +- * HashAggregate (76) - : +- Exchange (75) - : +- * HashAggregate (74) - : +- * Project (73) - : +- * BroadcastHashJoin Inner BuildRight (72) - : :- * Project (70) - : : +- * BroadcastHashJoin Inner BuildRight (69) - : : :- * ColumnarToRow (67) - : : : +- CometFilter (66) - : : : +- CometScan parquet spark_catalog.default.web_returns (65) - : : +- ReusedExchange (68) - : +- ReusedExchange (71) - :- * HashAggregate (88) - : +- Exchange (87) - : +- * HashAggregate (86) - : +- * HashAggregate (85) - : +- ReusedExchange (84) - +- * HashAggregate (93) - +- Exchange (92) - +- * HashAggregate (91) - +- * HashAggregate (90) - +- ReusedExchange (89) +TakeOrderedAndProject (99) ++- * HashAggregate (98) + +- Exchange (97) + +- * HashAggregate (96) + +- Union (95) + :- * HashAggregate (84) + : +- Exchange (83) + : +- * HashAggregate (82) + : +- Union (81) + : :- * Project (32) + : : +- * BroadcastHashJoin LeftOuter BuildRight (31) + : : :- * HashAggregate (17) + : : : +- Exchange (16) + : : : +- * ColumnarToRow (15) + : : : +- CometHashAggregate (14) + : : : +- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometProject (5) + : : : : +- CometFilter (4) + : : : : +- CometScan parquet spark_catalog.default.date_dim (3) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.store (9) + : : +- BroadcastExchange (30) + : : +- * HashAggregate (29) + : : +- Exchange (28) + : : +- * ColumnarToRow (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.store_returns (18) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : :- * Project (51) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (50) + : : :- BroadcastExchange (41) + : : : +- * HashAggregate (40) + : : : +- Exchange (39) + : : : +- * ColumnarToRow (38) + : : : +- CometHashAggregate (37) + : : : +- CometProject (36) + : : : +- CometBroadcastHashJoin (35) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (33) + : : : +- ReusedExchange (34) + : : +- * HashAggregate (49) + : : +- Exchange (48) + : : +- * ColumnarToRow (47) + : : +- CometHashAggregate (46) + : : +- CometProject (45) + : : +- CometBroadcastHashJoin (44) + : : :- CometScan parquet spark_catalog.default.catalog_returns (42) + : : +- ReusedExchange (43) + : +- * Project (80) + : +- * BroadcastHashJoin LeftOuter BuildRight (79) + : :- * HashAggregate (65) + : : +- Exchange (64) + : : +- * ColumnarToRow (63) + : : +- CometHashAggregate (62) + : : +- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (56) + : : : +- CometBroadcastHashJoin (55) + : : : :- CometFilter (53) + : : : : +- CometScan parquet spark_catalog.default.web_sales (52) + : : : +- ReusedExchange (54) + : : +- CometBroadcastExchange (59) + : : +- CometFilter (58) + : : +- CometScan parquet spark_catalog.default.web_page (57) + : +- BroadcastExchange (78) + : +- * HashAggregate (77) + : +- Exchange (76) + : +- * ColumnarToRow (75) + : +- CometHashAggregate (74) + : +- CometProject (73) + : +- CometBroadcastHashJoin (72) + : :- CometProject (70) + : : +- CometBroadcastHashJoin (69) + : : :- CometFilter (67) + : : : +- CometScan parquet spark_catalog.default.web_returns (66) + : : +- ReusedExchange (68) + : +- ReusedExchange (71) + :- * HashAggregate (89) + : +- Exchange (88) + : +- * HashAggregate (87) + : +- * HashAggregate (86) + : +- ReusedExchange (85) + +- * HashAggregate (94) + +- Exchange (93) + +- * HashAggregate (92) + +- * HashAggregate (91) + +- ReusedExchange (90) (1) Scan parquet spark_catalog.default.store_sales @@ -111,522 +112,510 @@ ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#6] +(4) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#6, d_date#7] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(6) CometBroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: [d_date_sk#6] -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +(7) CometBroadcastHashJoin +Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Right output [1]: [d_date_sk#6] +Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight + +(8) CometProject Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] +Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -(7) Scan parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] +(9) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) - -(9) ColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] +(10) CometFilter +Input [1]: [s_store_sk#8] +Condition : isnotnull(s_store_sk#8) -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [1]: [s_store_sk#8] +Arguments: [s_store_sk#8] -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] +Right output [1]: [s_store_sk#8] +Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +(13) CometProject +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] +(14) CometHashAggregate +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] +Keys [1]: [s_store_sk#8] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] -(14) Exchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(15) ColumnarToRow [codegen id : 1] +Input [3]: [s_store_sk#8, sum#9, sum#10] + +(16) Exchange +Input [3]: [s_store_sk#8, sum#9, sum#10] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] -(15) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] +(17) HashAggregate [codegen id : 4] +Input [3]: [s_store_sk#8, sum#9, sum#10] +Keys [1]: [s_store_sk#8] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#11, sum(UnscaledValue(ss_net_profit#3))#12] +Results [3]: [s_store_sk#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS sales#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#12,17,2) AS profit#14] -(16) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +(18) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#19)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(17) CometFilter -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) +(19) CometFilter +Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] +Condition : isnotnull(sr_store_sk#15) -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +(20) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#20] -(19) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#21] +(21) CometBroadcastHashJoin +Left output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] +Right output [1]: [d_date_sk#20] +Arguments: [sr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight -(20) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None +(22) CometProject +Input [5]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18, d_date_sk#20] +Arguments: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17], [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] -(21) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] +(23) ReusedExchange [Reuses operator id: 11] +Output [1]: [s_store_sk#21] -(22) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#22] +(24) CometBroadcastHashJoin +Left output [3]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] +Right output [1]: [s_store_sk#21] +Arguments: [sr_store_sk#15], [s_store_sk#21], Inner, BuildRight -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None +(25) CometProject +Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, s_store_sk#21] +Arguments: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21], [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] + +(26) CometHashAggregate +Input [3]: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] +Keys [1]: [s_store_sk#21] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#16)), partial_sum(UnscaledValue(sr_net_loss#17))] -(24) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] - -(25) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] -Keys [1]: [s_store_sk#22] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#23, sum#24] -Results [3]: [s_store_sk#22, sum#25, sum#26] - -(26) Exchange -Input [3]: [s_store_sk#22, sum#25, sum#26] -Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] - -(27) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#22, sum#25, sum#26] -Keys [1]: [s_store_sk#22] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] -Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] - -(28) BroadcastExchange -Input [3]: [s_store_sk#22, returns#29, profit_loss#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#22] +(27) ColumnarToRow [codegen id : 2] +Input [3]: [s_store_sk#21, sum#22, sum#23] + +(28) Exchange +Input [3]: [s_store_sk#21, sum#22, sum#23] +Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(29) HashAggregate [codegen id : 3] +Input [3]: [s_store_sk#21, sum#22, sum#23] +Keys [1]: [s_store_sk#21] +Functions [2]: [sum(UnscaledValue(sr_return_amt#16)), sum(UnscaledValue(sr_net_loss#17))] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#16))#24, sum(UnscaledValue(sr_net_loss#17))#25] +Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#24,17,2) AS returns#26, MakeDecimal(sum(UnscaledValue(sr_net_loss#17))#25,17,2) AS profit_loss#27] + +(30) BroadcastExchange +Input [3]: [s_store_sk#21, returns#26, profit_loss#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(31) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [s_store_sk#8] +Right keys [1]: [s_store_sk#21] Join type: LeftOuter Join condition: None -(30) Project [codegen id : 8] -Output [5]: [store channel AS channel#31, s_store_sk#7 AS id#32, sales#14, coalesce(returns#29, 0.00) AS returns#33, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#34] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] +(32) Project [codegen id : 4] +Output [5]: [store channel AS channel#28, s_store_sk#8 AS id#29, sales#13, coalesce(returns#26, 0.00) AS returns#30, (profit#14 - coalesce(profit_loss#27, 0.00)) AS profit#31] +Input [6]: [s_store_sk#8, sales#13, profit#14, s_store_sk#21, returns#26, profit_loss#27] -(31) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(33) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] +PartitionFilters: [isnotnull(cs_sold_date_sk#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +(34) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#37] -(33) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#40] +(35) CometBroadcastHashJoin +Left output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] +Right output [1]: [d_date_sk#37] +Arguments: [cs_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None +(36) CometProject +Input [5]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35, d_date_sk#37] +Arguments: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34], [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] -(35) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] - -(36) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum#41, sum#42] -Results [3]: [cs_call_center_sk#35, sum#43, sum#44] - -(37) Exchange -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] - -(38) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#35, sum#43, sum#44] -Keys [1]: [cs_call_center_sk#35] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] -Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] - -(39) BroadcastExchange -Input [3]: [cs_call_center_sk#35, sales#47, profit#48] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(40) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(37) CometHashAggregate +Input [3]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] +Keys [1]: [cs_call_center_sk#32] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#33)), partial_sum(UnscaledValue(cs_net_profit#34))] + +(38) ColumnarToRow [codegen id : 5] +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] + +(39) Exchange +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +Arguments: hashpartitioning(cs_call_center_sk#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(40) HashAggregate [codegen id : 6] +Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +Keys [1]: [cs_call_center_sk#32] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#33)), sum(UnscaledValue(cs_net_profit#34))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#33))#40, sum(UnscaledValue(cs_net_profit#34))#41] +Results [3]: [cs_call_center_sk#32, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#33))#40,17,2) AS sales#42, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#41,17,2) AS profit#43] + +(41) BroadcastExchange +Input [3]: [cs_call_center_sk#32, sales#42, profit#43] +Arguments: IdentityBroadcastMode, [plan_id=5] + +(42) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] +PartitionFilters: [isnotnull(cr_returned_date_sk#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] ReadSchema: struct -(41) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +(43) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#48] -(42) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#53] +(44) CometBroadcastHashJoin +Left output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +Right output [1]: [d_date_sk#48] +Arguments: [cr_returned_date_sk#46], [d_date_sk#48], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#51] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None +(45) CometProject +Input [4]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46, d_date_sk#48] +Arguments: [cr_return_amount#44, cr_net_loss#45], [cr_return_amount#44, cr_net_loss#45] -(44) Project [codegen id : 13] -Output [2]: [cr_return_amount#49, cr_net_loss#50] -Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] - -(45) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#49, cr_net_loss#50] +(46) CometHashAggregate +Input [2]: [cr_return_amount#44, cr_net_loss#45] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum#54, sum#55] -Results [2]: [sum#56, sum#57] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#44)), partial_sum(UnscaledValue(cr_net_loss#45))] + +(47) ColumnarToRow [codegen id : 7] +Input [2]: [sum#49, sum#50] -(46) Exchange -Input [2]: [sum#56, sum#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] +(48) Exchange +Input [2]: [sum#49, sum#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(47) HashAggregate -Input [2]: [sum#56, sum#57] +(49) HashAggregate +Input [2]: [sum#49, sum#50] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] +Functions [2]: [sum(UnscaledValue(cr_return_amount#44)), sum(UnscaledValue(cr_net_loss#45))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#44))#51, sum(UnscaledValue(cr_net_loss#45))#52] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#44))#51,17,2) AS returns#53, MakeDecimal(sum(UnscaledValue(cr_net_loss#45))#52,17,2) AS profit_loss#54] -(48) BroadcastNestedLoopJoin [codegen id : 14] +(50) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(49) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#62, cs_call_center_sk#35 AS id#63, sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#64] -Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] +(51) Project [codegen id : 8] +Output [5]: [catalog channel AS channel#55, cs_call_center_sk#32 AS id#56, sales#42, returns#53, (profit#43 - profit_loss#54) AS profit#57] +Input [5]: [cs_call_center_sk#32, sales#42, profit#43, returns#53, profit_loss#54] -(50) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(52) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(51) CometFilter -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_web_page_sk#65) - -(52) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +(53) CometFilter +Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] +Condition : isnotnull(ws_web_page_sk#58) -(53) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#70] +(54) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#63] -(54) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#68] -Right keys [1]: [d_date_sk#70] -Join type: Inner -Join condition: None +(55) CometBroadcastHashJoin +Left output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] +Right output [1]: [d_date_sk#63] +Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner, BuildRight -(55) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] -Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] +(56) CometProject +Input [5]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61, d_date_sk#63] +Arguments: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60], [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] -(56) Scan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#71] +(57) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct -(57) CometFilter -Input [1]: [wp_web_page_sk#71] -Condition : isnotnull(wp_web_page_sk#71) +(58) CometFilter +Input [1]: [wp_web_page_sk#64] +Condition : isnotnull(wp_web_page_sk#64) -(58) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#71] +(59) CometBroadcastExchange +Input [1]: [wp_web_page_sk#64] +Arguments: [wp_web_page_sk#64] -(59) BroadcastExchange -Input [1]: [wp_web_page_sk#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] +(60) CometBroadcastHashJoin +Left output [3]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] +Right output [1]: [wp_web_page_sk#64] +Arguments: [ws_web_page_sk#58], [wp_web_page_sk#64], Inner, BuildRight -(60) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#65] -Right keys [1]: [wp_web_page_sk#71] -Join type: Inner -Join condition: None +(61) CometProject +Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +Arguments: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64], [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] + +(62) CometHashAggregate +Input [3]: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +Keys [1]: [wp_web_page_sk#64] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#59)), partial_sum(UnscaledValue(ws_net_profit#60))] + +(63) ColumnarToRow [codegen id : 9] +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] -(61) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] - -(62) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum#72, sum#73] -Results [3]: [wp_web_page_sk#71, sum#74, sum#75] - -(63) Exchange -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=9] - -(64) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#71, sum#74, sum#75] -Keys [1]: [wp_web_page_sk#71] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] -Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] - -(65) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(64) Exchange +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +Arguments: hashpartitioning(wp_web_page_sk#64, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(65) HashAggregate [codegen id : 12] +Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +Keys [1]: [wp_web_page_sk#64] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#59)), sum(UnscaledValue(ws_net_profit#60))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#59))#67, sum(UnscaledValue(ws_net_profit#60))#68] +Results [3]: [wp_web_page_sk#64, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#67,17,2) AS sales#69, MakeDecimal(sum(UnscaledValue(ws_net_profit#60))#68,17,2) AS profit#70] + +(66) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(wr_returned_date_sk#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(66) CometFilter -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] -Condition : isnotnull(wr_web_page_sk#80) - -(67) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +(67) CometFilter +Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +Condition : isnotnull(wr_web_page_sk#71) -(68) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#85] +(68) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#76] -(69) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#83] -Right keys [1]: [d_date_sk#85] -Join type: Inner -Join condition: None +(69) CometBroadcastHashJoin +Left output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +Right output [1]: [d_date_sk#76] +Arguments: [wr_returned_date_sk#74], [d_date_sk#76], Inner, BuildRight -(70) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] -Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] +(70) CometProject +Input [5]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74, d_date_sk#76] +Arguments: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73], [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] (71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#86] - -(72) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#80] -Right keys [1]: [wp_web_page_sk#86] -Join type: Inner -Join condition: None - -(73) Project [codegen id : 20] -Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] - -(74) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum#87, sum#88] -Results [3]: [wp_web_page_sk#86, sum#89, sum#90] - -(75) Exchange -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, [plan_id=10] - -(76) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#86, sum#89, sum#90] -Keys [1]: [wp_web_page_sk#86] -Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] -Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] - -(77) BroadcastExchange -Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(78) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#71] -Right keys [1]: [wp_web_page_sk#86] +Output [1]: [wp_web_page_sk#77] + +(72) CometBroadcastHashJoin +Left output [3]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] +Right output [1]: [wp_web_page_sk#77] +Arguments: [wr_web_page_sk#71], [wp_web_page_sk#77], Inner, BuildRight + +(73) CometProject +Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +Arguments: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77], [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] + +(74) CometHashAggregate +Input [3]: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#72)), partial_sum(UnscaledValue(wr_net_loss#73))] + +(75) ColumnarToRow [codegen id : 10] +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] + +(76) Exchange +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(77) HashAggregate [codegen id : 11] +Input [3]: [wp_web_page_sk#77, sum#78, sum#79] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(wr_return_amt#72)), sum(UnscaledValue(wr_net_loss#73))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#72))#80, sum(UnscaledValue(wr_net_loss#73))#81] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(wr_return_amt#72))#80,17,2) AS returns#82, MakeDecimal(sum(UnscaledValue(wr_net_loss#73))#81,17,2) AS profit_loss#83] + +(78) BroadcastExchange +Input [3]: [wp_web_page_sk#77, returns#82, profit_loss#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +(79) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [wp_web_page_sk#64] +Right keys [1]: [wp_web_page_sk#77] Join type: LeftOuter Join condition: None -(79) Project [codegen id : 22] -Output [5]: [web channel AS channel#95, wp_web_page_sk#71 AS id#96, sales#78, coalesce(returns#93, 0.00) AS returns#97, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#98] -Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] - -(80) Union - -(81) HashAggregate [codegen id : 23] -Input [5]: [channel#31, id#32, sales#14, returns#33, profit#34] -Keys [2]: [channel#31, id#32] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#33), partial_sum(profit#34)] -Aggregate Attributes [6]: [sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(82) Exchange -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, [plan_id=12] - -(83) HashAggregate [codegen id : 24] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [5]: [channel#31, id#32, cast(sum(sales#14)#111 as decimal(37,2)) AS sales#114, cast(sum(returns#33)#112 as decimal(37,2)) AS returns#115, cast(sum(profit#34)#113 as decimal(38,2)) AS profit#116] - -(84) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(85) HashAggregate [codegen id : 48] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [4]: [channel#31, sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] - -(86) HashAggregate [codegen id : 48] -Input [4]: [channel#31, sales#117, returns#118, profit#119] -Keys [1]: [channel#31] -Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] -Aggregate Attributes [6]: [sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Results [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(87) Exchange -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, [plan_id=13] - -(88) HashAggregate [codegen id : 49] -Input [7]: [channel#31, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Keys [1]: [channel#31] -Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] -Aggregate Attributes [3]: [sum(sales#117)#132, sum(returns#118)#133, sum(profit#119)#134] -Results [5]: [channel#31, null AS id#135, sum(sales#117)#132 AS sales#136, sum(returns#118)#133 AS returns#137, sum(profit#119)#134 AS profit#138] - -(89) ReusedExchange [Reuses operator id: 82] -Output [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] - -(90) HashAggregate [codegen id : 73] -Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] -Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] -Results [3]: [sum(sales#14)#111 AS sales#117, sum(returns#33)#112 AS returns#118, sum(profit#34)#113 AS profit#119] - -(91) HashAggregate [codegen id : 73] -Input [3]: [sales#117, returns#118, profit#119] +(80) Project [codegen id : 12] +Output [5]: [web channel AS channel#84, wp_web_page_sk#64 AS id#85, sales#69, coalesce(returns#82, 0.00) AS returns#86, (profit#70 - coalesce(profit_loss#83, 0.00)) AS profit#87] +Input [6]: [wp_web_page_sk#64, sales#69, profit#70, wp_web_page_sk#77, returns#82, profit_loss#83] + +(81) Union + +(82) HashAggregate [codegen id : 13] +Input [5]: [channel#28, id#29, sales#13, returns#30, profit#31] +Keys [2]: [channel#28, id#29] +Functions [3]: [partial_sum(sales#13), partial_sum(returns#30), partial_sum(profit#31)] +Aggregate Attributes [6]: [sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93] +Results [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] + +(83) Exchange +Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Arguments: hashpartitioning(channel#28, id#29, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(84) HashAggregate [codegen id : 14] +Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [2]: [channel#28, id#29] +Functions [3]: [sum(sales#13), sum(returns#30), sum(profit#31)] +Aggregate Attributes [3]: [sum(sales#13)#100, sum(returns#30)#101, sum(profit#31)#102] +Results [5]: [channel#28, id#29, cast(sum(sales#13)#100 as decimal(37,2)) AS sales#103, cast(sum(returns#30)#101 as decimal(37,2)) AS returns#104, cast(sum(profit#31)#102 as decimal(38,2)) AS profit#105] + +(85) ReusedExchange [Reuses operator id: 83] +Output [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] + +(86) HashAggregate [codegen id : 28] +Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [2]: [channel#28, id#29] +Functions [3]: [sum(sales#13), sum(returns#30), sum(profit#31)] +Aggregate Attributes [3]: [sum(sales#13)#100, sum(returns#30)#101, sum(profit#31)#102] +Results [4]: [channel#28, sum(sales#13)#100 AS sales#106, sum(returns#30)#101 AS returns#107, sum(profit#31)#102 AS profit#108] + +(87) HashAggregate [codegen id : 28] +Input [4]: [channel#28, sales#106, returns#107, profit#108] +Keys [1]: [channel#28] +Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] +Aggregate Attributes [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Results [7]: [channel#28, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] + +(88) Exchange +Input [7]: [channel#28, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] +Arguments: hashpartitioning(channel#28, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(89) HashAggregate [codegen id : 29] +Input [7]: [channel#28, sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] +Keys [1]: [channel#28] +Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] +Aggregate Attributes [3]: [sum(sales#106)#121, sum(returns#107)#122, sum(profit#108)#123] +Results [5]: [channel#28, null AS id#124, sum(sales#106)#121 AS sales#125, sum(returns#107)#122 AS returns#126, sum(profit#108)#123 AS profit#127] + +(90) ReusedExchange [Reuses operator id: 83] +Output [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] + +(91) HashAggregate [codegen id : 43] +Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [2]: [channel#28, id#29] +Functions [3]: [sum(sales#13), sum(returns#30), sum(profit#31)] +Aggregate Attributes [3]: [sum(sales#13)#100, sum(returns#30)#101, sum(profit#31)#102] +Results [3]: [sum(sales#13)#100 AS sales#106, sum(returns#30)#101 AS returns#107, sum(profit#31)#102 AS profit#108] + +(92) HashAggregate [codegen id : 43] +Input [3]: [sales#106, returns#107, profit#108] Keys: [] -Functions [3]: [partial_sum(sales#117), partial_sum(returns#118), partial_sum(profit#119)] -Aggregate Attributes [6]: [sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -Results [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] +Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] +Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Results [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -(92) Exchange -Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] +(93) Exchange +Input [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(93) HashAggregate [codegen id : 74] -Input [6]: [sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] +(94) HashAggregate [codegen id : 44] +Input [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] Keys: [] -Functions [3]: [sum(sales#117), sum(returns#118), sum(profit#119)] -Aggregate Attributes [3]: [sum(sales#117)#151, sum(returns#118)#152, sum(profit#119)#153] -Results [5]: [null AS channel#154, null AS id#155, sum(sales#117)#151 AS sales#156, sum(returns#118)#152 AS returns#157, sum(profit#119)#153 AS profit#158] +Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] +Aggregate Attributes [3]: [sum(sales#106)#140, sum(returns#107)#141, sum(profit#108)#142] +Results [5]: [null AS channel#143, null AS id#144, sum(sales#106)#140 AS sales#145, sum(returns#107)#141 AS returns#146, sum(profit#108)#142 AS profit#147] -(94) Union +(95) Union -(95) HashAggregate [codegen id : 75] -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(96) HashAggregate [codegen id : 45] +Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +Keys [5]: [channel#28, id#29, sales#103, returns#104, profit#105] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Results [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -(96) Exchange -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: hashpartitioning(channel#31, id#32, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, [plan_id=15] +(97) Exchange +Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +Arguments: hashpartitioning(channel#28, id#29, sales#103, returns#104, profit#105, 5), ENSURE_REQUIREMENTS, [plan_id=13] -(97) HashAggregate [codegen id : 76] -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +(98) HashAggregate [codegen id : 46] +Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +Keys [5]: [channel#28, id#29, sales#103, returns#104, profit#105] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Results [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -(98) TakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] -Arguments: 100, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#114, returns#115, profit#116] +(99) TakeOrderedAndProject +Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +Arguments: 100, [channel#28 ASC NULLS FIRST, id#29 ASC NULLS FIRST], [channel#28, id#29, sales#103, returns#104, profit#105] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (103) -+- * ColumnarToRow (102) - +- CometProject (101) - +- CometFilter (100) - +- CometScan parquet spark_catalog.default.date_dim (99) +BroadcastExchange (104) ++- * ColumnarToRow (103) + +- CometProject (102) + +- CometFilter (101) + +- CometScan parquet spark_catalog.default.date_dim (100) -(99) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#159] +(100) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_date#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(100) CometFilter -Input [2]: [d_date_sk#6, d_date#159] -Condition : (((isnotnull(d_date#159) AND (d_date#159 >= 1998-08-04)) AND (d_date#159 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) +(101) CometFilter +Input [2]: [d_date_sk#6, d_date#7] +Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) -(101) CometProject -Input [2]: [d_date_sk#6, d_date#159] +(102) CometProject +Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(102) ColumnarToRow [codegen id : 1] +(103) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(103) BroadcastExchange +(104) BroadcastExchange Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#35 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 40 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 65 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt index 670a7e6c3e..752b8c8541 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt @@ -1,34 +1,34 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (76) + WholeStageCodegen (46) HashAggregate [channel,id,sales,returns,profit] InputAdapter Exchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) + WholeStageCodegen (45) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (24) + WholeStageCodegen (14) HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Exchange [channel,id] #2 - WholeStageCodegen (23) + WholeStageCodegen (13) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (8) + WholeStageCodegen (4) Project [s_store_sk,sales,returns,profit,profit_loss] BroadcastHashJoin [s_store_sk,s_store_sk] HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] InputAdapter Exchange [s_store_sk] #3 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] + CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] CometFilter [ss_store_sk] CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -39,129 +39,116 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometScan parquet spark_catalog.default.store [s_store_sk] + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange #6 + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) + BroadcastExchange #7 + WholeStageCodegen (3) HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [s_store_sk] #7 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [s_store_sk] #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometHashAggregate [s_store_sk,sr_return_amt,sr_net_loss] + CometProject [sr_return_amt,sr_net_loss,s_store_sk] + CometBroadcastHashJoin [sr_store_sk,s_store_sk] + CometProject [sr_store_sk,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [sr_returned_date_sk,d_date_sk] CometFilter [sr_store_sk] CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) + ReusedExchange [d_date_sk] #5 + ReusedExchange [s_store_sk] #6 + WholeStageCodegen (8) Project [cs_call_center_sk,sales,returns,profit,profit_loss] BroadcastNestedLoopJoin InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) + BroadcastExchange #9 + WholeStageCodegen (6) HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [cs_call_center_sk] #9 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [cs_call_center_sk] #10 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometHashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + CometBroadcastHashJoin [cs_sold_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 + ReusedExchange [d_date_sk] #5 HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange #10 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange #11 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometHashAggregate [cr_return_amount,cr_net_loss] + CometProject [cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cr_returned_date_sk,d_date_sk] CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (12) Project [wp_web_page_sk,sales,returns,profit,profit_loss] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #11 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [wp_web_page_sk] #12 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometHashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometProject [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_web_page_sk] CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - ColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange #13 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) + BroadcastExchange #14 + WholeStageCodegen (11) HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - Exchange [wp_web_page_sk] #14 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + Exchange [wp_web_page_sk] #15 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometHashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] + CometProject [wr_return_amt,wr_net_loss,wp_web_page_sk] + CometBroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + CometProject [wr_web_page_sk,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [wr_returned_date_sk,d_date_sk] CometFilter [wr_web_page_sk] CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) + ReusedExchange [d_date_sk] #5 + ReusedExchange [wp_web_page_sk] #13 + WholeStageCodegen (29) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange [channel] #15 - WholeStageCodegen (48) + Exchange [channel] #16 + WholeStageCodegen (28) HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) + WholeStageCodegen (44) HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter - Exchange #16 - WholeStageCodegen (73) + Exchange #17 + WholeStageCodegen (43) HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt index ff41ac064e..e125bab59b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt @@ -1,38 +1,40 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * Project (33) - +- Window (32) - +- * Sort (31) - +- Exchange (30) - +- * HashAggregate (29) - +- Exchange (28) - +- * HashAggregate (27) - +- Union (26) - :- * HashAggregate (15) - : +- Exchange (14) - : +- * HashAggregate (13) - : +- * Project (12) - : +- * BroadcastHashJoin Inner BuildRight (11) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * ColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (10) - : +- * ColumnarToRow (9) - : +- CometFilter (8) - : +- CometScan parquet spark_catalog.default.item (7) - :- * HashAggregate (20) - : +- Exchange (19) - : +- * HashAggregate (18) - : +- * HashAggregate (17) - : +- ReusedExchange (16) - +- * HashAggregate (25) - +- Exchange (24) - +- * HashAggregate (23) - +- * HashAggregate (22) - +- ReusedExchange (21) +TakeOrderedAndProject (36) ++- * Project (35) + +- Window (34) + +- * Sort (33) + +- Exchange (32) + +- * HashAggregate (31) + +- Exchange (30) + +- * HashAggregate (29) + +- Union (28) + :- * HashAggregate (17) + : +- Exchange (16) + : +- * ColumnarToRow (15) + : +- CometHashAggregate (14) + : +- CometProject (13) + : +- CometBroadcastHashJoin (12) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (11) + : +- CometFilter (10) + : +- CometScan parquet spark_catalog.default.item (9) + :- * HashAggregate (22) + : +- Exchange (21) + : +- * HashAggregate (20) + : +- * HashAggregate (19) + : +- ReusedExchange (18) + +- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * HashAggregate (24) + +- ReusedExchange (23) (1) Scan parquet spark_catalog.default.web_sales @@ -47,194 +49,203 @@ ReadSchema: struct Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +(3) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct -(4) ReusedExchange [Reuses operator id: 39] -Output [1]: [d_date_sk#5] +(4) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None +(5) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] +Arguments: [d_date_sk#5], [d_date_sk#5] -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] +(6) CometBroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: [d_date_sk#5] + +(7) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +Right output [1]: [d_date_sk#5] +Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight + +(8) CometProject Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] +Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] -(7) Scan parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] +(9) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#7, i_class#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) - -(9) ColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#7, i_category#8] +(10) CometFilter +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Condition : isnotnull(i_item_sk#7) -(10) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(11) CometBroadcastExchange +Input [3]: [i_item_sk#7, i_class#8, i_category#9] +Arguments: [i_item_sk#7, i_class#8, i_category#9] -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None +(12) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#1, ws_net_paid#2] +Right output [3]: [i_item_sk#7, i_class#8, i_category#9] +Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight -(12) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_class#7, i_category#8] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#7, i_category#8] +(13) CometProject +Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#8, i_category#9] +Arguments: [ws_net_paid#2, i_class#8, i_category#9], [ws_net_paid#2, i_class#8, i_category#9] -(13) HashAggregate [codegen id : 3] -Input [3]: [ws_net_paid#2, i_class#7, i_category#8] -Keys [2]: [i_category#8, i_class#7] +(14) CometHashAggregate +Input [3]: [ws_net_paid#2, i_class#8, i_category#9] +Keys [2]: [i_category#9, i_class#8] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#9] -Results [3]: [i_category#8, i_class#7, sum#10] -(14) Exchange -Input [3]: [i_category#8, i_class#7, sum#10] -Arguments: hashpartitioning(i_category#8, i_class#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] +(15) ColumnarToRow [codegen id : 1] +Input [3]: [i_category#9, i_class#8, sum#10] -(15) HashAggregate [codegen id : 4] -Input [3]: [i_category#8, i_class#7, sum#10] -Keys [2]: [i_category#8, i_class#7] +(16) Exchange +Input [3]: [i_category#9, i_class#8, sum#10] +Arguments: hashpartitioning(i_category#9, i_class#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(17) HashAggregate [codegen id : 2] +Input [3]: [i_category#9, i_class#8, sum#10] +Keys [2]: [i_category#9, i_class#8] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) as decimal(27,2)) AS total_sum#12, i_category#8, i_class#7, 0 AS g_category#13, 0 AS g_class#14, 0 AS lochierarchy#15] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) as decimal(27,2)) AS total_sum#12, i_category#9, i_class#8, 0 AS g_category#13, 0 AS g_class#14, 0 AS lochierarchy#15] -(16) ReusedExchange [Reuses operator id: 14] -Output [3]: [i_category#8, i_class#7, sum#16] +(18) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#9, i_class#8, sum#16] -(17) HashAggregate [codegen id : 8] -Input [3]: [i_category#8, i_class#7, sum#16] -Keys [2]: [i_category#8, i_class#7] +(19) HashAggregate [codegen id : 4] +Input [3]: [i_category#9, i_class#8, sum#16] +Keys [2]: [i_category#9, i_class#8] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) AS total_sum#17, i_category#8] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) AS total_sum#17, i_category#9] -(18) HashAggregate [codegen id : 8] -Input [2]: [total_sum#17, i_category#8] -Keys [1]: [i_category#8] +(20) HashAggregate [codegen id : 4] +Input [2]: [total_sum#17, i_category#9] +Keys [1]: [i_category#9] Functions [1]: [partial_sum(total_sum#17)] Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [3]: [i_category#8, sum#20, isEmpty#21] +Results [3]: [i_category#9, sum#20, isEmpty#21] -(19) Exchange -Input [3]: [i_category#8, sum#20, isEmpty#21] -Arguments: hashpartitioning(i_category#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] +(21) Exchange +Input [3]: [i_category#9, sum#20, isEmpty#21] +Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(20) HashAggregate [codegen id : 9] -Input [3]: [i_category#8, sum#20, isEmpty#21] -Keys [1]: [i_category#8] +(22) HashAggregate [codegen id : 5] +Input [3]: [i_category#9, sum#20, isEmpty#21] +Keys [1]: [i_category#9] Functions [1]: [sum(total_sum#17)] Aggregate Attributes [1]: [sum(total_sum#17)#22] -Results [6]: [sum(total_sum#17)#22 AS total_sum#23, i_category#8, null AS i_class#24, 0 AS g_category#25, 1 AS g_class#26, 1 AS lochierarchy#27] +Results [6]: [sum(total_sum#17)#22 AS total_sum#23, i_category#9, null AS i_class#24, 0 AS g_category#25, 1 AS g_class#26, 1 AS lochierarchy#27] -(21) ReusedExchange [Reuses operator id: 14] -Output [3]: [i_category#8, i_class#7, sum#28] +(23) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#9, i_class#8, sum#28] -(22) HashAggregate [codegen id : 13] -Input [3]: [i_category#8, i_class#7, sum#28] -Keys [2]: [i_category#8, i_class#7] +(24) HashAggregate [codegen id : 7] +Input [3]: [i_category#9, i_class#8, sum#28] +Keys [2]: [i_category#9, i_class#8] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) AS total_sum#17] -(23) HashAggregate [codegen id : 13] +(25) HashAggregate [codegen id : 7] Input [1]: [total_sum#17] Keys: [] Functions [1]: [partial_sum(total_sum#17)] Aggregate Attributes [2]: [sum#29, isEmpty#30] Results [2]: [sum#31, isEmpty#32] -(24) Exchange +(26) Exchange Input [2]: [sum#31, isEmpty#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(25) HashAggregate [codegen id : 14] +(27) HashAggregate [codegen id : 8] Input [2]: [sum#31, isEmpty#32] Keys: [] Functions [1]: [sum(total_sum#17)] Aggregate Attributes [1]: [sum(total_sum#17)#33] Results [6]: [sum(total_sum#17)#33 AS total_sum#34, null AS i_category#35, null AS i_class#36, 1 AS g_category#37, 1 AS g_class#38, 2 AS lochierarchy#39] -(26) Union +(28) Union -(27) HashAggregate [codegen id : 15] -Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +(29) HashAggregate [codegen id : 9] +Input [6]: [total_sum#12, i_category#9, i_class#8, g_category#13, g_class#14, lochierarchy#15] +Keys [6]: [total_sum#12, i_category#9, i_class#8, g_category#13, g_class#14, lochierarchy#15] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +Results [6]: [total_sum#12, i_category#9, i_class#8, g_category#13, g_class#14, lochierarchy#15] -(28) Exchange -Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -Arguments: hashpartitioning(total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15, 5), ENSURE_REQUIREMENTS, [plan_id=5] +(30) Exchange +Input [6]: [total_sum#12, i_category#9, i_class#8, g_category#13, g_class#14, lochierarchy#15] +Arguments: hashpartitioning(total_sum#12, i_category#9, i_class#8, g_category#13, g_class#14, lochierarchy#15, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(29) HashAggregate [codegen id : 16] -Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] -Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +(31) HashAggregate [codegen id : 10] +Input [6]: [total_sum#12, i_category#9, i_class#8, g_category#13, g_class#14, lochierarchy#15] +Keys [6]: [total_sum#12, i_category#9, i_class#8, g_category#13, g_class#14, lochierarchy#15] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, CASE WHEN (g_class#14 = 0) THEN i_category#8 END AS _w0#40] +Results [5]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, CASE WHEN (g_class#14 = 0) THEN i_category#9 END AS _w0#40] -(30) Exchange -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] -Arguments: hashpartitioning(lochierarchy#15, _w0#40, 5), ENSURE_REQUIREMENTS, [plan_id=6] +(32) Exchange +Input [5]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, _w0#40] +Arguments: hashpartitioning(lochierarchy#15, _w0#40, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(31) Sort [codegen id : 17] -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] +(33) Sort [codegen id : 11] +Input [5]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, _w0#40] Arguments: [lochierarchy#15 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#12 DESC NULLS LAST], false, 0 -(32) Window -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40] +(34) Window +Input [5]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, _w0#40] Arguments: [rank(total_sum#12) windowspecdefinition(lochierarchy#15, _w0#40, total_sum#12 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [lochierarchy#15, _w0#40], [total_sum#12 DESC NULLS LAST] -(33) Project [codegen id : 18] -Output [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] -Input [6]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#40, rank_within_parent#41] +(35) Project [codegen id : 12] +Output [5]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, rank_within_parent#41] +Input [6]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, _w0#40, rank_within_parent#41] -(34) TakeOrderedAndProject -Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] -Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#41] +(36) TakeOrderedAndProject +Input [5]: [total_sum#12, i_category#9, i_class#8, lochierarchy#15, rank_within_parent#41] +Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#12, i_category#9, i_class#8, lochierarchy#15, rank_within_parent#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (39) -+- * ColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan parquet spark_catalog.default.date_dim (35) +BroadcastExchange (41) ++- * ColumnarToRow (40) + +- CometProject (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.date_dim (37) -(35) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#42] +(37) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(36) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#42] -Condition : (((isnotnull(d_month_seq#42) AND (d_month_seq#42 >= 1212)) AND (d_month_seq#42 <= 1223)) AND isnotnull(d_date_sk#5)) +(38) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#6] +Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(37) CometProject -Input [2]: [d_date_sk#5, d_month_seq#42] +(39) CometProject +Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(38) ColumnarToRow [codegen id : 1] +(40) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(39) BroadcastExchange +(41) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index 5c5e088857..0b7ad47269 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -1,32 +1,32 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (18) + WholeStageCodegen (12) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (17) + WholeStageCodegen (11) Sort [lochierarchy,_w0,total_sum] InputAdapter Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (16) + WholeStageCodegen (10) HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] [_w0] InputAdapter Exchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (15) + WholeStageCodegen (9) HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] InputAdapter Union - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,g_category,g_class,lochierarchy,sum] InputAdapter Exchange [i_category,i_class] #3 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_category,i_class,ws_net_paid] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,i_item_sk] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_sold_date_sk,d_date_sk] CometFilter [ws_item_sk] CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -37,29 +37,27 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometProject [d_date_sk] CometFilter [d_month_seq,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (9) + CometBroadcastExchange #5 + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange #6 + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (5) HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] InputAdapter - Exchange [i_category] #6 - WholeStageCodegen (8) + Exchange [i_category] #7 + WholeStageCodegen (4) HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] InputAdapter ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (14) + WholeStageCodegen (8) HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] InputAdapter - Exchange #7 - WholeStageCodegen (13) + Exchange #8 + WholeStageCodegen (7) HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index f77b7ec93b..799687a197 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -1,25 +1,27 @@ == Physical Plan == -* Sort (21) -+- Exchange (20) - +- * Project (19) - +- Window (18) - +- * Sort (17) - +- Exchange (16) - +- * HashAggregate (15) - +- Exchange (14) - +- * HashAggregate (13) - +- * Project (12) - +- * BroadcastHashJoin Inner BuildRight (11) - :- * Project (9) - : +- * BroadcastHashJoin Inner BuildRight (8) - : :- * ColumnarToRow (3) - : : +- CometFilter (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (7) - : +- * ColumnarToRow (6) - : +- CometFilter (5) - : +- CometScan parquet spark_catalog.default.item (4) - +- ReusedExchange (10) +* Sort (23) ++- Exchange (22) + +- * Project (21) + +- Window (20) + +- * Sort (19) + +- Exchange (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * ColumnarToRow (15) + +- CometHashAggregate (14) + +- CometProject (13) + +- CometBroadcastHashJoin (12) + :- CometProject (7) + : +- CometBroadcastHashJoin (6) + : :- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (5) + : +- CometFilter (4) + : +- CometScan parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (11) + +- CometProject (10) + +- CometFilter (9) + +- CometScan parquet spark_catalog.default.date_dim (8) (1) Scan parquet spark_catalog.default.store_sales @@ -34,122 +36,131 @@ ReadSchema: struct Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) -(3) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(4) Scan parquet spark_catalog.default.item +(3) Scan parquet spark_catalog.default.item Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(6) ColumnarToRow [codegen id : 1] +(5) CometBroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(7) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] +(6) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Right output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(7) CometProject Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(10) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#11] +(8) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None +(9) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(12) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +(10) CometProject +Input [2]: [d_date_sk#11, d_date#12] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(11) CometBroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: [d_date_sk#11] + +(12) CometBroadcastHashJoin +Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Right output [1]: [d_date_sk#11] +Arguments: [ss_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight + +(13) CometProject Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] +Arguments: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) HashAggregate [codegen id : 3] +(14) CometHashAggregate Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#12] -Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -(14) Exchange +(15) ColumnarToRow [codegen id : 1] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(15) HashAggregate [codegen id : 4] +(16) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16] -(16) Exchange +(18) Exchange Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] -Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(17) Sort [codegen id : 5] +(19) Sort [codegen id : 3] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: [i_class#9 ASC NULLS FIRST], false, 0 -(18) Window +(20) Window Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] -(19) Project [codegen id : 6] +(21) Project [codegen id : 4] Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] -(20) Exchange +(22) Exchange Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] -Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(21) Sort [codegen id : 7] +(23) Sort [codegen id : 5] Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], true, 0 ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (26) -+- * ColumnarToRow (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan parquet spark_catalog.default.date_dim (22) +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) -(22) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#19] +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(23) CometFilter -Input [2]: [d_date_sk#11, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) +(25) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(24) CometProject -Input [2]: [d_date_sk#11, d_date#19] +(26) CometProject +Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(25) ColumnarToRow [codegen id : 1] +(27) ColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(26) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index b7489a0aff..6484c2dcb2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -1,27 +1,27 @@ -WholeStageCodegen (7) +WholeStageCodegen (5) Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] InputAdapter Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) + WholeStageCodegen (4) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) + WholeStageCodegen (3) Sort [i_class] InputAdapter Exchange [i_class] #2 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] InputAdapter Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - ColumnarToRow - InputAdapter + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_sold_date_sk,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,i_item_sk] CometFilter [ss_item_sk] CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 @@ -32,12 +32,10 @@ WholeStageCodegen (7) CometProject [d_date_sk] CometFilter [d_date,d_date_sk] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [i_category,i_item_sk] - CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange #5 + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange #6 + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala index 91d88c76e8..8bae2ecaa3 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala @@ -121,6 +121,7 @@ class CometJoinSuite extends CometTestBase { test("HashJoin without join filter") { withSQLConf( + "spark.sql.join.forceApplyShuffledHashJoin" -> "true", SQLConf.PREFER_SORTMERGEJOIN.key -> "false", SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { @@ -149,24 +150,35 @@ class CometJoinSuite extends CometTestBase { // Left join: build left // sql("SELECT /*+ SHUFFLE_HASH(tbl_a) */ * FROM tbl_a LEFT JOIN tbl_b ON tbl_a._2 = tbl_b._1") - // TODO: DataFusion HashJoin doesn't support build right yet. // Inner join: build right - // sql("SELECT /*+ SHUFFLE_HASH(tbl_b) */ * FROM tbl_a JOIN tbl_b ON tbl_a._2 = tbl_b._1") - // + val df4 = + sql( + "SELECT /*+ SHUFFLE_HASH(tbl_b) */ * FROM tbl_a JOIN tbl_b ON tbl_a._2 = tbl_b._1") + checkSparkAnswerAndOperator(df4) + // Left join: build right - // sql("SELECT /*+ SHUFFLE_HASH(tbl_b) */ * FROM tbl_a LEFT JOIN tbl_b ON tbl_a._2 = tbl_b._1") - // + val df5 = + sql("SELECT /*+ SHUFFLE_HASH(tbl_b) */ * FROM tbl_a LEFT JOIN tbl_b ON tbl_a._2 = tbl_b._1") + checkSparkAnswerAndOperator(df5) + // Right join: build right - // sql("SELECT /*+ SHUFFLE_HASH(tbl_b) */ * FROM tbl_a RIGHT JOIN tbl_b ON tbl_a._2 = tbl_b._1") - // + val df6 = + sql("SELECT /*+ SHUFFLE_HASH(tbl_b) */ * FROM tbl_a RIGHT JOIN tbl_b ON tbl_a._2 = tbl_b._1") + checkSparkAnswerAndOperator(df6) + // Full join: build right - // sql("SELECT /*+ SHUFFLE_HASH(tbl_b) */ * FROM tbl_a FULL JOIN tbl_b ON tbl_a._2 = tbl_b._1") - // - // val left = sql("SELECT * FROM tbl_a") - // val right = sql("SELECT * FROM tbl_b") - // + val df7 = + sql("SELECT /*+ SHUFFLE_HASH(tbl_b) */ * FROM tbl_a FULL JOIN tbl_b ON tbl_a._2 = tbl_b._1") + checkSparkAnswerAndOperator(df7) + // Left semi and anti joins are only supported with build right in Spark. - // left.join(right, left("_2") === right("_1"), "leftsemi") + val left = sql("SELECT * FROM tbl_a") + val right = sql("SELECT * FROM tbl_b") + + val df8 = left.join(right, left("_2") === right("_1"), "leftsemi") + checkSparkAnswerAndOperator(df8) + + // DataFusion HashJoin LeftAnti has bugs in handling nulls and is disabled for now. // left.join(right, left("_2") === right("_1"), "leftanti") } }